From bbbd160ca7904a7e0a338183bfb5b213a3b7d323 Mon Sep 17 00:00:00 2001 From: xuyang Date: Thu, 2 Jan 2025 19:49:43 +0800 Subject: [PATCH] [FLINK-36277][table] Create a DynamicTableSource instead of registering a legacy TableSource in tests and remove TableEnvironmentInternal#registerTableSourceInternal --- .../api/internal/TableEnvironmentImpl.java | 4 +- .../internal/TableEnvironmentInternal.java | 13 - .../utils/InputPriorityGraphGenerator.java | 4 +- .../logical/FlinkRewriteSubQueryRule.scala | 6 +- .../rules/logical/SplitAggregateRule.scala | 3 +- .../org.apache.flink.table.factories.Factory | 1 + .../flink/table/api/batch/ExplainTest.xml | 12 +- .../flink/table/api/stream/ExplainTest.xml | 12 +- .../table/planner/plan/batch/sql/CalcTest.xml | 100 +- .../plan/batch/sql/DagOptimizationTest.xml | 204 +-- .../plan/batch/sql/DeadlockBreakupTest.xml | 86 +- .../planner/plan/batch/sql/LimitTest.xml | 32 +- .../batch/sql/MultipleInputCreationTest.xml | 558 +++---- .../plan/batch/sql/PartitionableSinkTest.xml | 16 +- .../table/planner/plan/batch/sql/RankTest.xml | 32 +- .../plan/batch/sql/SetOperatorsTest.xml | 76 +- .../planner/plan/batch/sql/SortLimitTest.xml | 32 +- .../table/planner/plan/batch/sql/SortTest.xml | 20 +- .../plan/batch/sql/SubplanReuseTest.xml | 244 +-- .../planner/plan/batch/sql/TableScanTest.xml | 16 - .../planner/plan/batch/sql/UnionTest.xml | 20 +- .../planner/plan/batch/sql/UnnestTest.xml | 32 +- .../batch/sql/WindowTableFunctionTest.xml | 36 +- .../batch/sql/agg/DistinctAggregateTest.xml | 100 +- .../plan/batch/sql/agg/GroupWindowTest.xml | 258 +-- .../plan/batch/sql/agg/GroupingSetsTest.xml | 146 +- .../plan/batch/sql/agg/HashAggregateTest.xml | 168 +- .../plan/batch/sql/agg/OverAggregateTest.xml | 104 +- .../plan/batch/sql/agg/SortAggregateTest.xml | 216 +-- .../batch/sql/join/BroadcastHashJoinTest.xml | 190 +-- .../join/BroadcastHashSemiAntiJoinTest.xml | 620 +++---- .../batch/sql/join/NestedLoopJoinTest.xml | 302 ++-- .../sql/join/NestedLoopSemiAntiJoinTest.xml | 828 ++++----- .../plan/batch/sql/join/SemiAntiJoinTest.xml | 812 ++++----- .../batch/sql/join/ShuffledHashJoinTest.xml | 222 +-- .../sql/join/ShuffledHashSemiAntiJoinTest.xml | 622 +++---- .../plan/batch/sql/join/SingleRowJoinTest.xml | 58 +- .../plan/batch/sql/join/SortMergeJoinTest.xml | 222 +-- .../sql/join/SortMergeSemiAntiJoinTest.xml | 622 +++---- .../plan/batch/table/AggregateTest.xml | 16 +- .../planner/plan/batch/table/CalcTest.xml | 52 +- .../plan/batch/table/ColumnFunctionsTest.xml | 4 +- .../plan/batch/table/CorrelateTest.xml | 28 +- .../plan/batch/table/GroupWindowTest.xml | 24 +- .../planner/plan/batch/table/JoinTest.xml | 104 +- .../plan/batch/table/PythonAggregateTest.xml | 8 +- .../plan/batch/table/PythonCalcTest.xml | 4 +- .../table/PythonGroupWindowAggregateTest.xml | 12 +- .../table/PythonOverWindowAggregateTest.xml | 8 +- .../plan/batch/table/SetOperatorsTest.xml | 54 +- .../planner/plan/common/PartialInsertTest.xml | 132 +- .../CalcPruneAggregateCallRuleTest.xml | 68 +- .../CalcPythonCorrelateTransposeRuleTest.xml | 4 +- .../logical/CalcRankTransposeRuleTest.xml | 12 +- .../logical/ConvertToNotInOrInRuleTest.xml | 100 +- .../logical/DecomposeGroupingSetsRuleTest.xml | 32 +- .../logical/ExpressionReductionRulesTest.xml | 28 +- ...regateExpandDistinctAggregatesRuleTest.xml | 108 +- .../FlinkAggregateJoinTransposeRuleTest.xml | 68 +- .../logical/FlinkAggregateRemoveRuleTest.xml | 96 +- .../rules/logical/FlinkCalcMergeRuleTest.xml | 36 +- .../logical/FlinkJoinToMultiJoinRuleTest.xml | 388 ++--- .../logical/FlinkLimit0RemoveRuleTest.xml | 36 +- ...inkLogicalRankRuleForConstantRangeTest.xml | 60 +- .../FlinkLogicalRankRuleForRangeEndTest.xml | 52 +- .../logical/FlinkPruneEmptyRulesTest.xml | 10 +- ...inkSemiAntiJoinFilterTransposeRuleTest.xml | 16 +- ...FlinkSemiAntiJoinJoinTransposeRuleTest.xml | 432 ++--- ...nkSemiAntiJoinProjectTransposeRuleTest.xml | 24 +- .../JoinConditionEqualityTransferRuleTest.xml | 112 +- .../JoinConditionTypeCoerceRuleTest.xml | 74 +- ...inDependentConditionDerivationRuleTest.xml | 68 +- .../logical/JoinPushExpressionsRuleTest.xml | 32 +- ...rrelateToJoinFromTemporalTableRuleTest.xml | 18 +- .../rules/logical/LogicalUnnestRuleTest.xml | 32 +- .../ProjectPruneAggregateCallRuleTest.xml | 68 +- .../ProjectSemiAntiJoinTransposeRuleTest.xml | 148 +- .../rules/logical/PythonCalcSplitRuleTest.xml | 108 +- .../logical/PythonCorrelateSplitRuleTest.xml | 16 +- .../rules/logical/PythonMapMergeRuleTest.xml | 12 +- .../logical/RemoveSingleAggregateRuleTest.xml | 8 +- .../ReplaceIntersectWithSemiJoinRuleTest.xml | 32 +- .../ReplaceMinusWithAntiJoinRuleTest.xml | 32 +- .../logical/RewriteIntersectAllRuleTest.xml | 32 +- .../rules/logical/RewriteMinusAllRuleTest.xml | 32 +- .../RewriteMultiJoinConditionRuleTest.xml | 168 +- .../SimplifyFilterConditionRuleTest.xml | 76 +- .../logical/SimplifyJoinConditionRuleTest.xml | 34 +- .../rules/logical/SplitAggregateRuleTest.xml | 76 +- ...itPythonConditionFromCorrelateRuleTest.xml | 4 +- .../SplitPythonConditionFromJoinRuleTest.xml | 16 +- .../logical/WindowGroupReorderRuleTest.xml | 40 +- .../subquery/FlinkRewriteSubQueryRuleTest.xml | 152 +- .../logical/subquery/SubQueryAntiJoinTest.xml | 756 ++++----- .../logical/subquery/SubQuerySemiJoinTest.xml | 1480 ++++++++--------- ...bqueryCorrelateVariablesValidationTest.xml | 30 +- .../batch/EnforceLocalHashAggRuleTest.xml | 24 +- .../batch/EnforceLocalSortAggRuleTest.xml | 36 +- .../RemoveRedundantLocalHashAggRuleTest.xml | 16 +- .../RemoveRedundantLocalRankRuleTest.xml | 12 +- .../RemoveRedundantLocalSortAggRuleTest.xml | 20 +- .../planner/plan/stream/sql/CalcTest.xml | 100 +- .../plan/stream/sql/DagOptimizationTest.xml | 332 ++-- .../stream/sql/NonDeterministicDagTest.xml | 32 +- .../plan/stream/sql/PartitionableSinkTest.xml | 16 +- .../planner/plan/stream/sql/RankTest.xml | 4 +- .../plan/stream/sql/SetOperatorsTest.xml | 76 +- .../plan/stream/sql/SubplanReuseTest.xml | 156 +- .../planner/plan/stream/sql/TableScanTest.xml | 16 - .../planner/plan/stream/sql/UnionTest.xml | 20 +- .../planner/plan/stream/sql/UnnestTest.xml | 36 +- .../plan/stream/sql/agg/AggregateTest.xml | 114 +- .../stream/sql/agg/DistinctAggregateTest.xml | 288 ++-- .../plan/stream/sql/agg/GroupingSetsTest.xml | 144 +- .../sql/agg/IncrementalAggregateTest.xml | 72 +- .../stream/sql/agg/TwoStageAggregateTest.xml | 28 +- .../planner/plan/stream/sql/join/JoinTest.xml | 354 ++-- .../plan/stream/sql/join/SemiAntiJoinTest.xml | 792 ++++----- .../plan/stream/table/AggregateTest.xml | 48 +- .../planner/plan/stream/table/CalcTest.xml | 36 +- .../plan/stream/table/ColumnFunctionsTest.xml | 52 +- .../plan/stream/table/CorrelateTest.xml | 68 +- .../table/GroupWindowTableAggregateTest.xml | 24 +- .../plan/stream/table/GroupWindowTest.xml | 9 +- .../plan/stream/table/PythonAggregateTest.xml | 12 +- .../plan/stream/table/PythonCalcTest.xml | 4 +- .../table/PythonGroupWindowAggregateTest.xml | 50 +- .../table/PythonOverWindowAggregateTest.xml | 30 +- .../stream/table/PythonTableAggregateTest.xml | 12 +- .../plan/stream/table/SetOperatorsTest.xml | 44 +- .../plan/stream/table/TableAggregateTest.xml | 52 +- .../stream/table/TwoStageAggregateTest.xml | 24 +- .../validation/AggregateValidationTest.xml | 4 +- .../planner/plan/batch/sql/CalcTest.scala | 7 +- .../plan/batch/sql/SetOperatorsTest.scala | 12 +- .../plan/batch/sql/TableScanTest.scala | 6 - .../batch/sql/WindowTableFunctionTest.scala | 3 +- .../batch/sql/agg/AggregateTestBase.scala | 32 +- .../plan/batch/table/SetOperatorsTest.scala | 12 +- .../common/DistinctAggregateTestBase.scala | 5 +- .../planner/plan/common/UnnestTestBase.scala | 8 +- .../DecomposeGroupingSetsRuleTest.scala | 5 +- .../FlinkAggregateJoinTransposeRuleTest.scala | 21 +- .../FlinkAggregateRemoveRuleTest.scala | 41 +- .../logical/FlinkCalcMergeRuleTest.scala | 4 +- .../PruneAggregateCallRuleTestBase.scala | 21 +- ...ueryCorrelateVariablesValidationTest.scala | 9 +- .../planner/plan/stream/sql/CalcTest.scala | 5 +- .../plan/stream/sql/SetOperatorsTest.scala | 12 +- .../plan/stream/sql/TableScanTest.scala | 6 - .../plan/stream/sql/agg/AggregateTest.scala | 32 +- .../validation/OverWindowValidationTest.scala | 2 +- .../table/planner/utils/TableTestBase.scala | 234 ++- .../planner/utils/testTableSourceSinks.scala | 11 - 154 files changed, 8285 insertions(+), 8081 deletions(-) diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java index b8b116f9bcccd..5f809f52ac3ca 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java @@ -1347,8 +1347,8 @@ protected List> translate(List modifyOperatio return planner.translate(modifyOperations); } - @Override - public void registerTableSourceInternal(String name, TableSource tableSource) { + /** TODO FLINK-36132 Remove this method later. */ + private void registerTableSourceInternal(String name, TableSource tableSource) { validateTableSource(tableSource); ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier(UnresolvedIdentifier.of(name)); diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentInternal.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentInternal.java index 4f75c38763fb0..8d4f1f848c9ab 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentInternal.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentInternal.java @@ -108,19 +108,6 @@ default String explainInternal(List operations, ExplainDetail... extr String explainInternal( List operations, ExplainFormat format, ExplainDetail... extraDetails); - /** - * Registers an external {@link TableSource} in this {@link TableEnvironment}'s catalog. - * Registered tables can be referenced in SQL queries. - * - *

Temporary objects can shadow permanent ones. If a permanent object in a given path exists, - * it will be inaccessible in the current session. To make the permanent object available again - * one can drop the corresponding temporary object. - * - * @param name The name under which the {@link TableSource} is registered. - * @param tableSource The {@link TableSource} to register. - */ - void registerTableSourceInternal(String name, TableSource tableSource); - /** * Registers an external {@link TableSink} with already configured field names and field types * in this {@link TableEnvironment}'s catalog. Registered sink tables can be referenced in SQL diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityGraphGenerator.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityGraphGenerator.java index d31bd62eae518..c9556079b0a3f 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityGraphGenerator.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityGraphGenerator.java @@ -62,12 +62,12 @@ * : : +- Exchange(distribution=[hash[a]]) * : : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(*) AS count1$0]) * : : +- Calc(select=[a]) - * : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + * : : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) * : +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count1$0) AS cnt2], reuse_id=[1]) * : +- Exchange(distribution=[hash[d]]) * : +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(*) AS count1$0]) * : +- Calc(select=[d]) - * : +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + * : +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) * +- Calc(select=[cnt1, CAST(cnt2) AS cnt2]) * +- HashJoin(joinType=[LeftOuterJoin], where=[=(d, a)], select=[d, cnt2, a, cnt1], build=[right]) * :- Reused(reference_id=[1]) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkRewriteSubQueryRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkRewriteSubQueryRule.scala index 3cee597cd1a05..49388c70ee572 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkRewriteSubQueryRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkRewriteSubQueryRule.scala @@ -21,7 +21,7 @@ import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelOptRuleOperand} import org.apache.calcite.plan.RelOptRule.{any, operandJ} import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.core.{Aggregate, Filter, RelFactories} -import org.apache.calcite.rex.{RexShuttle, _} +import org.apache.calcite.rex._ import org.apache.calcite.sql.`type`.SqlTypeFamily import org.apache.calcite.sql.SqlKind import org.apache.calcite.sql.fun.SqlCountAggFunction @@ -39,11 +39,11 @@ import scala.collection.JavaConversions._ * {{{ * LogicalProject(a=[$0], b=[$1], c=[$2]) * +- LogicalJoin(condition=[$3], joinType=[semi]) - * :- LogicalTableScan(table=[[x, source: [TestTableSource(a, b, c)]]]) + * :- LogicalTableScan(table=[[x]]) * +- LogicalProject($f0=[IS NOT NULL($0)]) * +- LogicalAggregate(group=[{}], m=[MIN($0)]) * +- LogicalProject(i=[true]) - * +- LogicalTableScan(table=[[y, source: [TestTableSource(d, e, f)]]]) + * +- LogicalTableScan(table=[[y]]) * }}} */ class FlinkRewriteSubQueryRule( diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRule.scala index 1e8caa75d1706..d37a7f36275bf 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRule.scala @@ -81,8 +81,7 @@ import scala.collection.JavaConversions._ * +- FlinkLogicalExpand(projects=[a, b, c, $f3, $f4, $e]) * +- FlinkLogicalCalc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, * MOD(HASH_CODE(c), 1024) AS $f4]) - * +- FlinkLogicalTableSourceScan(table=[[MyTable, - * source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + * +- FlinkLogicalTableSourceScan(table=[[MyTable]], fields=[a, b, c]) * }}} * * '$e = 1' is equivalent to 'group by a, hash(b) % 1024' '$e = 2' is equivalent to 'group by a, diff --git a/flink-table/flink-table-planner/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-table/flink-table-planner/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory index 2444832afb032..4a0e2319b3a9d 100644 --- a/flink-table/flink-table-planner/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-table/flink-table-planner/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -21,3 +21,4 @@ org.apache.flink.table.planner.plan.stream.sql.TestTableFactory org.apache.flink.table.planner.factories.TestUpdateDeleteTableFactory org.apache.flink.table.planner.factories.TestSupportsStagingTableFactory org.apache.flink.table.planner.factories.TestProcedureCatalogFactory +org.apache.flink.table.planner.utils.TestSimpleDynamicTableSourceFactory \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml index 478af9e6f567c..bde61626c8563 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml @@ -450,13 +450,13 @@ SortLimit(orderBy=[a ASC], offset=[0], fetch=[5], global=[true]) @@ -464,13 +464,13 @@ LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml index e72ee7cf088be..b040e92375539 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml @@ -72,13 +72,13 @@ DataStreamScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, @@ -86,13 +86,13 @@ LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml index 69692846ffd63..b7ab117c336e2 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml @@ -23,13 +23,13 @@ limitations under the License. @@ -42,14 +42,14 @@ Calc(select=[ARRAY('Hi', 'Hello', c) AS EXPR$0]) LogicalProject(a=[$0], a1=[$1]) +- LogicalFilter(condition=[>($1, 10)]) +- LogicalProject(a=[$0], a1=[random_udf($1)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 10)]) +- Calc(select=[a, random_udf(b) AS a1]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -62,13 +62,13 @@ Calc(select=[a, a1], where=[(a1 > 10)]) LogicalProject(a=[$0]) +- LogicalFilter(condition=[>(random_udf($1), 10)]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> (random_udf(b), 10)]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -90,7 +90,7 @@ LogicalProject(a=[$0], r=[$1]) +- LogicalProject(a=[$0], r=[random_udf($1)]) +- LogicalProject(a=[$0], b=[$1], c1=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableFunctionScan(invocation=[str_split($cor0.c)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) ]]> @@ -99,7 +99,7 @@ LogicalProject(a=[$0], r=[$1]) Calc(select=[a, r], where=[>(r, 10)]) +- Calc(select=[a, random_udf(b) AS r]) +- Correlate(invocation=[str_split($cor0.c)], correlate=[table(str_split($cor0.c))], select=[a,b,c,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, VARCHAR(2147483647) EXPR$0)], joinType=[INNER]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -128,13 +128,13 @@ Calc(select=[a AS EXPR$0, CAST(b AS VARCHAR(2147483647)) AS EXPR$1]) ($1, 20))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 20))]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -180,13 +180,13 @@ Calc(select=[MAP('a', 0.12, 'b', 0.5) AS EXPR$0]) ($0, 20))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -201,13 +201,13 @@ WHERE b = DATE '1984-07-12' AND c = TIME '14:34:24' AND d = TIMESTAMP '1984-07-1 @@ -219,13 +219,13 @@ Calc(select=[a, CAST(1984-07-12 AS DATE) AS b, CAST(14:34:24 AS TIME(0)) AS c, C @@ -237,13 +237,13 @@ Calc(select=[a, b, CAST('xx' AS VARCHAR(2147483647)) AS c], where=[(SEARCH(b, Sa @@ -255,13 +255,13 @@ Calc(select=[a, b, CAST('xx' AS VARCHAR(2147483647)) AS c], where=[((SEARCH(b, S @@ -275,13 +275,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[<($0, 50)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>($1, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 0) AND (a < 50))]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -292,13 +292,13 @@ Calc(select=[a, b, c], where=[((b > 0) AND (a < 50))]) @@ -310,13 +310,13 @@ Calc(select=[a._1 AS _1, a._2 AS _2, c, b._1 AS _10, b._2 AS _20]) @@ -327,13 +327,13 @@ Calc(select=[c]) @@ -345,13 +345,13 @@ Calc(select=[MAP(a, c) AS EXPR$0]) @@ -363,13 +363,13 @@ Calc(select=[a, b, c], where=[SEARCH(a, Sarg[1, 10; NULL AS TRUE])]) @@ -381,13 +381,13 @@ Calc(select=[a, b, c], where=[(SEARCH(b, Sarg[(-∞..1), (1..3), (3..4), (4..5), ($1, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 0)]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -398,13 +398,13 @@ Calc(select=[a, b, c], where=[(b > 0)]) @@ -416,13 +416,13 @@ Calc(select=[a, c]) ($1, 2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 2)]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -433,13 +433,13 @@ Calc(select=[a, (b + 1) AS EXPR$1], where=[(b > 2)]) @@ -450,12 +450,12 @@ Calc(select=[IF(SEARCH(c, Sarg[_UTF-16LE''; NULL AS TRUE]), 'a', 'b') AS EXPR$0] @@ -466,13 +466,13 @@ LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable4, sourc @@ -489,13 +489,13 @@ FROM MyTable @@ -507,12 +507,12 @@ Calc(select=[a, b, c, 1984-07-12 AS EXPR$3, 14:34:24 AS EXPR$4, 1984-07-12 14:34 @@ -523,13 +523,13 @@ LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.xml index 381620b844456..21b02dd3bf0e8 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.xml @@ -22,7 +22,7 @@ limitations under the License. LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a, b, c]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink2`], fields=[b, cnt]) +- LogicalProject(b=[$0], cnt=[$1]) @@ -33,11 +33,11 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink2`], fields=[b :- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink3`], fields=[b, cnt]) +- LogicalProject(b=[$0], cnt=[$1]) @@ -48,16 +48,16 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink3`], fields=[b :- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -111,7 +111,7 @@ Calc(select=[sum_a])(reuse_id=[1]) +- Exchange(distribution=[hash[c]]) +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(a) AS sum$0]) +- Calc(select=[c, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) LegacySink(name=[`default_catalog`.`default_database`.`sink1`], fields=[total_sum]) +- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS total_sum]) @@ -135,36 +135,36 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a :- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b1=[$1]) +- LogicalProject(a=[$2], b1=[$1]) +- LogicalFilter(condition=[=($2, $0)]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, b1]) +- LogicalUnion(all=[true]) :- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b1=[$1]) +- LogicalProject(a=[$2], b1=[$1]) +- LogicalFilter(condition=[=($2, $0)]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> =($1, 5)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>=($0, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a1, b, c1]) +- LogicalProject(a1=[$2], b=[$3], c1=[$1]) @@ -259,23 +259,23 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a : +- LogicalFilter(condition=[<($1, 5)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalFilter(condition=[>=($0, 0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a1=[$0], b=[$1], c2=[$3]) +- LogicalFilter(condition=[=($0, $2)]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a1=[$0], b=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a2=[$0], c=[$2]) +- LogicalFilter(condition=[>=($1, 5)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>=($0, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> = 0)])(reuse_id=[2]) +- Reused(reference_id=[1]) @@ -311,19 +311,19 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink1`], fields=[t +- LogicalAggregate(group=[{}], total_sum=[SUM($0)]) +- LogicalProject(a=[$0]) +- LogicalProject(a=[random_udf($0)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink2`], fields=[total_min]) +- LogicalAggregate(group=[{}], total_min=[MIN($0)]) +- LogicalProject(a=[$0]) +- LogicalProject(a=[random_udf($0)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -366,10 +366,10 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink2`], fields=[t Union(all=[true], union=[a])(reuse_id=[1]) :- Calc(select=[a]) : +- Calc(select=[a, c]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) +- Calc(select=[d AS a]) +- Calc(select=[d, f]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[d, e, f]) LegacySink(name=[`default_catalog`.`default_database`.`sink1`], fields=[total_sum]) +- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS total_sum]) @@ -394,11 +394,11 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink1`], fields=[t +- LogicalUnion(all=[true]) :- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], c=[$2]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalProject(d=[$0], f=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(a=[$0], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink2`], fields=[total_min]) +- LogicalAggregate(group=[{}], total_min=[MIN($0)]) @@ -406,19 +406,19 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink2`], fields=[t +- LogicalUnion(all=[true]) :- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], c=[$2]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalProject(d=[$0], f=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(a=[$0], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink3`], fields=[a]) +- LogicalProject(a=[$0]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(d=[$0], f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -426,15 +426,15 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink3`], fields=[a Union(all=[true], union=[a])(reuse_id=[2]) :- Calc(select=[a]) : +- Calc(select=[a, c]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) +- Calc(select=[d AS a]) +- Calc(select=[d, f]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[d, e, f]) Union(all=[true], union=[a])(reuse_id=[1]) :- Reused(reference_id=[2]) +- Calc(select=[a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c]) LegacySink(name=[`default_catalog`.`default_database`.`sink1`], fields=[total_sum]) +- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS total_sum]) @@ -459,9 +459,9 @@ LegacySink(name=[`default_catalog`.`default_database`.`sink3`], fields=[a]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a, c]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(d=[$0], f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink2`], fields=[total_sum]) +- LogicalAggregate(group=[{}], total_sum=[SUM($0)]) @@ -470,11 +470,11 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink2`], fields=[t :- LogicalProject(a=[$0], c=[$1]) : +- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], c=[$2]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalProject(d=[$0], f=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(a=[$0], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink3`], fields=[total_min]) +- LogicalAggregate(group=[{}], total_min=[MIN($0)]) @@ -483,20 +483,20 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink3`], fields=[t :- LogicalProject(a=[$0], c=[$1]) : +- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], c=[$2]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalProject(d=[$0], f=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(a=[$0], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -556,13 +556,13 @@ Union(all=[true], union=[a])(reuse_id=[1]) :- Union(all=[true], union=[a]) : :- Calc(select=[a]) : : +- Calc(select=[a, c]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) : +- Calc(select=[d AS a]) : +- Calc(select=[d, f]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[d, e, f]) +- Calc(select=[a]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c]) LegacySink(name=[`default_catalog`.`default_database`.`sink1`], fields=[total_sum]) +- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS total_sum]) @@ -585,13 +585,13 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a +- LogicalProject(a=[$1], sum_c=[$2], time=[TUMBLE_END($0)], window_start=[TUMBLE_START($0)], window_end=[TUMBLE_END($0)]) +- LogicalAggregate(group=[{0, 1}], sum_c=[SUM($2)]) +- LogicalProject($f0=[$TUMBLE($3, 15000:INTERVAL SECOND)], a=[$0], $f2=[CAST($2):DOUBLE]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, sum_c, time]) +- LogicalProject(a=[$1], sum_c=[$2], time=[TUMBLE_END($0)]) +- LogicalAggregate(group=[{0, 1}], sum_c=[SUM($2)]) +- LogicalProject($f0=[$TUMBLE($3, 15000:INTERVAL SECOND)], a=[$0], $f2=[CAST($2):DOUBLE]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -600,7 +600,7 @@ HashWindowAggregate(groupBy=[a], window=[TumblingGroupWindow('w$, ts, 15000)], p +- Exchange(distribution=[hash[a]]) +- LocalHashWindowAggregate(groupBy=[a], window=[TumblingGroupWindow('w$, ts, 15000)], properties=[w$start, w$end, w$rowtime], select=[a, Partial_SUM(c) AS sum$0]) +- Calc(select=[ts, a, CAST(c AS DOUBLE) AS c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, ts)]]], fields=[a, b, c, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, ts]) LegacySink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a, sum_c, time, window_start, window_end]) +- Calc(select=[a, sum_c, w$end AS time, w$start AS window_start, w$end AS window_end]) @@ -626,12 +626,12 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a : +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) : :- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2]) : : +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalTableFunctionScan(invocation=[split($cor0.c)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) +- LogicalProject(a=[$0], total_c=[0]) +- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2]) +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, total_c]) +- LogicalProject(a=[$0], total_c=[$1]) @@ -644,17 +644,17 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a : +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) : :- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2]) : : +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalTableFunctionScan(invocation=[split($cor0.c)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) +- LogicalProject(a=[$0], total_c=[0]) +- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2]) +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -703,7 +703,7 @@ LegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[c, +- Exchange(distribution=[hash[c]]) +- LocalHashAggregate(groupBy=[c], select=[c, Partial_COUNT(a) AS count$0]) +- Calc(select=[c, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -718,18 +718,18 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fiel : +- LogicalFilter(condition=[<($1, 5)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalFilter(condition=[>=($0, 0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a1=[$0], b=[$1], c2=[$3]) +- LogicalFilter(condition=[=($0, $2)]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a1=[$0], b=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a2=[$0], c=[$2]) +- LogicalFilter(condition=[>=($1, 5)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>=($0, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -739,7 +739,7 @@ LegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a1, +- MultipleInput(readOrder=[2,1,0], members=[\nHashJoin(joinType=[InnerJoin], where=[(a1 = a3)], select=[a3, c1, a1, b], build=[right])\n:- [#1] Exchange(distribution=[hash[a3]])\n+- Calc(select=[a AS a1, b])\n +- HashJoin(joinType=[InnerJoin], where=[(a = a2)], select=[a, b, a2], build=[right])\n :- [#2] Exchange(distribution=[hash[a]])\n +- [#3] Exchange(distribution=[hash[a2]])\n]) :- Exchange(distribution=[hash[a3]]) : +- Calc(select=[a AS a3, c AS c1], where=[((a >= 0) AND (b < 5))]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])(reuse_id=[1]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b], where=[(a <= 10)]) : +- Reused(reference_id=[1]) @@ -757,14 +757,14 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fiel :- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b1=[$1]) +- LogicalProject(a=[$2], b1=[$1]) +- LogicalFilter(condition=[=($2, $0)]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -773,7 +773,7 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fiel LegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a1, b1]) +- Union(all=[true], union=[a, b]) :- Calc(select=[a, b], where=[(a <= 10)])(reuse_id=[1]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) +- Calc(select=[a0 AS a, b AS b1]) +- HashJoin(joinType=[InnerJoin], where=[(a0 = a)], select=[a, b, a0], build=[left]) :- Exchange(distribution=[hash[a]]) @@ -793,7 +793,7 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink`], fields=[a, +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a1=[$0], b=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a3=[$0], b1=[$3], c1=[$1]) +- LogicalFilter(condition=[=($2, $0)]) +- LogicalJoin(condition=[true], joinType=[inner]) @@ -801,18 +801,18 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink`], fields=[a, : +- LogicalFilter(condition=[<($1, 5)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalFilter(condition=[>=($0, 0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a1=[$0], b=[$1], c2=[$3]) +- LogicalFilter(condition=[=($0, $2)]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a1=[$0], b=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a2=[$0], c=[$2]) +- LogicalFilter(condition=[>=($1, 5)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>=($0, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -822,7 +822,7 @@ LegacySink(name=[`default_catalog`.`default_database`.`sink`], fields=[a, b, c]) +- MultipleInput(readOrder=[3,2,1,0], members=[\nHashJoin(joinType=[InnerJoin], where=[(a = a3)], select=[a, a3, b1, c1], build=[right])\n:- [#1] Exchange(distribution=[hash[a]])\n+- Calc(select=[a3, b AS b1, c1])\n +- HashJoin(joinType=[InnerJoin], where=[(a1 = a3)], select=[a3, c1, a1, b], build=[right])\n :- [#2] Exchange(distribution=[hash[a3]])\n +- Calc(select=[a AS a1, b])\n +- HashJoin(joinType=[InnerJoin], where=[(a = a2)], select=[a, b, a2], build=[right])\n :- [#3] Exchange(distribution=[hash[a]])\n +- [#4] Exchange(distribution=[hash[a2]])\n]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a], where=[(a <= 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])(reuse_id=[1]) :- Exchange(distribution=[hash[a3]]) : +- Calc(select=[a AS a3, c AS c1], where=[((a >= 0) AND (b < 5))]) : +- Reused(reference_id=[1]) @@ -845,9 +845,9 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink`], fields=[a, : +- LogicalFilter(condition=[AND(=($1, $4), =($0, $6))]) : +- LogicalJoin(condition=[true], joinType=[inner]) : :- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(i, j, k, l, m)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) +- LogicalTableFunctionScan(invocation=[split($cor0.c)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) ]]> @@ -859,11 +859,11 @@ LegacySink(name=[`default_catalog`.`default_database`.`sink`], fields=[a, b, c, :- Exchange(distribution=[hash[a]]) : +- HashJoin(joinType=[InnerJoin], where=[(b = e)], select=[a, b, c, d, e, f], build=[left]) : :- Exchange(distribution=[hash[b]]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) : +- Exchange(distribution=[hash[e]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[d, e, f]) +- Exchange(distribution=[hash[i]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(i, j, k, l, m)]]], fields=[i, j, k, l, m]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[i, j, k, l, m]) ]]> @@ -875,9 +875,9 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`sink`], fields=[to +- LogicalProject(a=[$0]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(d=[$0], f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -888,9 +888,9 @@ LegacySink(name=[`default_catalog`.`default_database`.`sink`], fields=[total_sum +- LocalHashAggregate(select=[Partial_SUM(a) AS sum$0]) +- Union(all=[true], union=[a]) :- Calc(select=[a]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) +- Calc(select=[d AS a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[d, e, f]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.xml index 745d53b628f45..e942d41a39cdf 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.xml @@ -105,10 +105,10 @@ LogicalProject(c=[$0], a=[$1], b=[$2], c0=[$3], a0=[$4], b0=[$5]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalAggregate(group=[{0}], a=[SUM($1)], b=[SUM($2)]) : +- LogicalProject(c=[$2], a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalAggregate(group=[{0}], a=[SUM($1)], b=[SUM($2)]) +- LogicalProject(c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -120,7 +120,7 @@ Calc(select=[c, a, b, c0, a1, b0]) : +- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_SUM(sum$0) AS a, Final_SUM(sum$1) AS b])(reuse_id=[1]) : +- Exchange(distribution=[hash[c]]) : +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(a) AS sum$0, Partial_SUM(b) AS sum$1]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[c, a, b], where=[(a > 1)]) +- Reused(reference_id=[1]) @@ -141,16 +141,16 @@ WHERE r1.a = r2.a and r1.b = 5 LogicalProject(a=[$0], b=[$4]) +- LogicalFilter(condition=[AND(=($0, $3), =($1, 5))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[IN($0, { LogicalProject(a=[$0]) LogicalFilter(condition=[>($1, 5)]) - LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + LogicalTableScan(table=[[default_catalog, default_database, x]]) })]) +- LogicalSort(fetch=[10]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -159,7 +159,7 @@ Calc(select=[a, b]) +- MultipleInput(readOrder=[2,0,1], members=[\nHashJoin(joinType=[InnerJoin], where=[(a = a0)], select=[a, a0, b], build=[right])\n:- [#1] Exchange(distribution=[hash[a]], shuffle_mode=[BATCH])\n+- HashJoin(joinType=[LeftSemiJoin], where=[(a = a0)], select=[a, b], build=[left])\n :- [#2] Exchange(distribution=[hash[a]])\n +- [#3] Exchange(distribution=[hash[a]], shuffle_mode=[BATCH])\n]) :- Exchange(distribution=[hash[a]], shuffle_mode=[BATCH]) : +- Calc(select=[a], where=[(b = 5)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c])(reuse_id=[1]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) : +- Limit(offset=[0], fetch=[10], global=[true]) @@ -186,10 +186,10 @@ LogicalProject(a=[$0]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalSort(fetch=[10]) : +- LogicalProject(a=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalSort(fetch=[10]) +- LogicalProject(a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -201,7 +201,7 @@ Calc(select=[a]) : +- Limit(offset=[0], fetch=[10], global=[true]) : +- Exchange(distribution=[single]) : +- Limit(offset=[0], fetch=[10], global=[false]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Reused(reference_id=[1]) ]]> @@ -221,10 +221,10 @@ LogicalProject(a=[$0]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalSort(fetch=[10]) : +- LogicalProject(a=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalSort(fetch=[10]) +- LogicalProject(a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -236,7 +236,7 @@ Calc(select=[a]) : +- Limit(offset=[0], fetch=[10], global=[true]) : +- Exchange(distribution=[single]) : +- Limit(offset=[0], fetch=[10], global=[false]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[any], shuffle_mode=[BATCH]) +- Reused(reference_id=[1]) ]]> @@ -260,12 +260,12 @@ LogicalProject(cnt=[$0], cnt0=[$1]) : +- LogicalFilter(condition=[>($0, 3)]) : +- LogicalAggregate(group=[{}], cnt=[COUNT()]) : +- LogicalProject($f0=[0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(cnt=[$0]) +- LogicalFilter(condition=[<($0, 5)]) +- LogicalAggregate(group=[{}], cnt=[COUNT()]) +- LogicalProject($f0=[0]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -276,7 +276,7 @@ NestedLoopJoin(joinType=[FullOuterJoin], where=[(cnt <> cnt0)], select=[cnt, cnt : +- Exchange(distribution=[single]) : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0]) : +- Calc(select=[0 AS $f0]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[single], shuffle_mode=[BATCH]) +- Calc(select=[cnt], where=[(cnt < 5)]) +- Reused(reference_id=[1]) @@ -303,18 +303,18 @@ LogicalUnion(all=[true]) : +- LogicalJoin(condition=[=($0, $2)], joinType=[left]) : :- LogicalAggregate(group=[{0}], cnt1=[COUNT()]) : : +- LogicalProject(a=[$0]) -: : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) +: : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalAggregate(group=[{0}], cnt2=[COUNT()]) : +- LogicalProject(d=[$0]) -: +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(cnt1=[$3], cnt2=[$1]) +- LogicalJoin(condition=[=($0, $2)], joinType=[left]) :- LogicalAggregate(group=[{0}], cnt2=[COUNT()]) : +- LogicalProject(d=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalAggregate(group=[{0}], cnt1=[COUNT()]) +- LogicalProject(a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -325,13 +325,13 @@ MultipleInput(readOrder=[0,2,1], members=[\nUnion(all=[true], union=[cnt1, cnt2] : +- Exchange(distribution=[hash[a]]) : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(*) AS count1$0]) : +- Calc(select=[a]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) :- Exchange(distribution=[hash[a]], shuffle_mode=[BATCH]) : +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[d]]) +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(*) AS count1$0]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) ]]> @@ -355,16 +355,16 @@ LogicalProject(a=[$0], b=[$1], c=[$2], a0=[$3], b0=[$4], c0=[$5], a1=[$6], b1=[$ : +- LogicalFilter(condition=[AND(=($0, $3), >($1, 10))]) : +- LogicalJoin(condition=[true], joinType=[inner]) : :- LogicalAggregate(group=[{0}], b=[SUM($1)], c=[MAX($2)]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalAggregate(group=[{0}], b=[SUM($1)], c=[MAX($2)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], a0=[$3], b0=[$4], c0=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), <($1, 5))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalAggregate(group=[{0}], b=[SUM($1)], c=[MAX($2)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalAggregate(group=[{0}], b=[SUM($1)], c=[MAX($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -382,7 +382,7 @@ HashJoin(joinType=[InnerJoin], where=[(c = c1)], select=[a, b, c, a0, b0, c0, a1 : : +- LocalSortAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_MAX(c) AS max$1]) : : +- Exchange(distribution=[forward]) : : +- Sort(orderBy=[a ASC]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) : +- Exchange(distribution=[hash[a]], shuffle_mode=[BATCH]) : +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[c]]) @@ -409,9 +409,9 @@ SELECT * FROM T1 LogicalProject(a=[$0], a0=[$1]) +- LogicalJoin(condition=[=($0, $1)], joinType=[inner]) :- LogicalProject(a=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -419,7 +419,7 @@ LogicalProject(a=[$0], a0=[$1]) HashJoin(joinType=[InnerJoin], where=[(a = a0)], select=[a, a0], build=[left]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a])(reuse_id=[1]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[a]], shuffle_mode=[BATCH]) +- Reused(reference_id=[1]) ]]> @@ -440,13 +440,13 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3], a0=[$4], b0=[$5], d0=[$6], e0=[$7 :- LogicalProject(a=[$0], b=[$1], d=[$3], e=[$4]) : +- LogicalFilter(condition=[=($0, $3)]) : +- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[$1], d=[$3], e=[$4]) +- LogicalFilter(condition=[=($0, $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -456,10 +456,10 @@ HashJoin(joinType=[InnerJoin], where=[(b = e0)], select=[a, b, d, e, a0, b0, d0, : +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[left]) : :- Exchange(distribution=[hash[a]]) : : +- Calc(select=[a, b], where=[(a < 10)]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])(reuse_id=[1]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c])(reuse_id=[1]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, e], where=[(d < 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])(reuse_id=[2]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f])(reuse_id=[2]) +- Exchange(distribution=[hash[e]], shuffle_mode=[BATCH]) +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[left]) :- Exchange(distribution=[hash[a]]) @@ -485,12 +485,12 @@ SELECT x.a, x.b, T1.* FROM x LEFT JOIN T1 ON x.a = T1.k WHERE x.a > 0 AND T1.v = LogicalProject(a=[$0], b=[$1], a0=[$3], b0=[$4], c=[$5], k=[$6], v=[$7]) +- LogicalFilter(condition=[AND(>($0, 0), =($7, 0))]) +- LogicalJoin(condition=[=($0, $6)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], k=[$3], v=[+($1, $4)]) +- LogicalFilter(condition=[>($3, 0)]) +- LogicalJoin(condition=[=($0, $3)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -499,7 +499,7 @@ Calc(select=[a, b, a0, b0, c, k, CAST(0 AS BIGINT) AS v]) +- HashJoin(joinType=[InnerJoin], where=[(a = k)], select=[a, b, a0, b0, c, k], build=[right]) :- Exchange(distribution=[hash[a]], shuffle_mode=[BATCH]) : +- Calc(select=[a, b], where=[(a > 0)])(reuse_id=[2]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c])(reuse_id=[1]) +- Exchange(distribution=[keep_input_as_is[hash[k]]]) +- Calc(select=[a, b, c, a0 AS k]) +- Exchange(distribution=[keep_input_as_is[hash[a0]]]) @@ -531,10 +531,10 @@ LogicalProject(a=[$0], b=[$1], c=[$2], a0=[$3], b0=[$4], c0=[$5]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a=[MAX($0) OVER (PARTITION BY $1 ORDER BY $1 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)], b=[$1], c=[$2]) : +- LogicalProject(a=[CASE(>(COUNT($0) OVER (PARTITION BY $1 ORDER BY $1 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $1 NULLS FIRST), null:INTEGER)], b=[$1], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[MIN($0) OVER (PARTITION BY $1 ORDER BY $1 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)], b=[$1], c=[$2]) +- LogicalProject(a=[CASE(>(COUNT($0) OVER (PARTITION BY $1 ORDER BY $1 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $1 NULLS FIRST), null:INTEGER)], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -550,7 +550,7 @@ HashJoin(joinType=[InnerJoin], where=[(c = c0)], select=[a, b, c, a0, b0, c0], b : +- Exchange(distribution=[forward]) : +- Sort(orderBy=[b ASC]) : +- Exchange(distribution=[hash[b]]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[c]], shuffle_mode=[BATCH]) +- Calc(select=[w0$o0 AS a, b, c]) +- OverAggregate(partitionBy=[b], orderBy=[b ASC], window#0=[MIN(a) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[a, b, c, w0$o0]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/LimitTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/LimitTest.xml index f12c8282b6393..e24be96ba551f 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/LimitTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/LimitTest.xml @@ -24,7 +24,7 @@ limitations under the License. @@ -61,7 +61,7 @@ Limit(offset=[0], fetch=[10], global=[true]) @@ -70,7 +70,7 @@ Calc(select=[a, c]) +- Limit(offset=[10], fetch=[10], global=[true]) +- Exchange(distribution=[single]) +- Limit(offset=[0], fetch=[20], global=[false]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -102,7 +102,7 @@ Limit(offset=[10], fetch=[10], global=[true]) @@ -111,7 +111,7 @@ Calc(select=[a, c]) +- Limit(offset=[0], fetch=[10], global=[true]) +- Exchange(distribution=[single]) +- Limit(offset=[0], fetch=[10], global=[false]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -123,7 +123,7 @@ Calc(select=[a, c]) @@ -140,7 +140,7 @@ Values(tuples=[[]], values=[a, c]) @@ -157,7 +157,7 @@ Values(tuples=[[]], values=[a, c]) @@ -214,7 +214,7 @@ SortLimit(orderBy=[c ASC], offset=[0], fetch=[10], global=[true]) @@ -223,7 +223,7 @@ Calc(select=[a, c]) +- Limit(offset=[1], fetch=[10], global=[true]) +- Exchange(distribution=[single]) +- Limit(offset=[0], fetch=[11], global=[false]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -235,7 +235,7 @@ Calc(select=[a, c]) @@ -244,7 +244,7 @@ Calc(select=[a, c]) +- Limit(offset=[0], fetch=[10], global=[true]) +- Exchange(distribution=[single]) +- Limit(offset=[0], fetch=[10], global=[false]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -256,7 +256,7 @@ Calc(select=[a, c]) @@ -264,7 +264,7 @@ LogicalSort(fetch=[5]) Limit(offset=[0], fetch=[5], global=[true]) +- Exchange(distribution=[single]) +- Limit(offset=[0], fetch=[5], global=[false]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -276,7 +276,7 @@ Limit(offset=[0], fetch=[5], global=[true]) @@ -284,7 +284,7 @@ LogicalSort(offset=[10]) Calc(select=[a, c]) +- Limit(offset=[10], fetch=[unlimited], global=[true]) +- Exchange(distribution=[single]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.xml index 46ed24ad97faf..ecef3e634e08f 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.xml @@ -31,8 +31,8 @@ LogicalProject(a=[$0], b=[$1], c=[$2], nx=[$3], d=[$4], e=[$5], f=[$6], ny=[$7], +- LogicalFilter(condition=[>($1, 10)]) +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -42,9 +42,9 @@ MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftOuterJo :- Exchange(distribution=[broadcast]) : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) :- Calc(select=[a, b, c, nx], where=[(b > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) +- Exchange(distribution=[broadcast]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) ]]> @@ -63,8 +63,8 @@ LogicalProject(a=[$0], b=[$1], c=[$2], nx=[$3], d=[$4], e=[$5], f=[$6], ny=[$7], +- LogicalFilter(condition=[>($1, 10)]) +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -74,9 +74,9 @@ MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftOuterJo :- Exchange(distribution=[broadcast]) : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) :- Calc(select=[a, b, c, nx], where=[(b > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) +- Exchange(distribution=[broadcast]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) ]]> @@ -95,8 +95,8 @@ LogicalProject(a=[$0], b=[$1], c=[$2], nx=[$3], d=[$4], e=[$5], f=[$6], ny=[$7], +- LogicalFilter(condition=[>($1, 10)]) +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -106,9 +106,9 @@ MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftOuterJo :- Exchange(distribution=[broadcast]) : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) :- Calc(select=[a, b, c, nx], where=[(b > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) +- Exchange(distribution=[broadcast]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) ]]> @@ -133,15 +133,15 @@ LogicalProject(a=[$0]) : :- LogicalProject(a=[$0]) : : +- LogicalProject(a=[$0]) : : +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) : +- LogicalProject(a=[$0]) : +- LogicalJoin(condition=[>($0, $3)], joinType=[full]) : :- LogicalTableScan(table=[[default_catalog, default_database, t]]) : +- LogicalAggregate(group=[{}], cnt=[COUNT()]) : +- LogicalProject($f0=[0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(g, h, i, nz)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(g, h, i, nz)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -150,7 +150,7 @@ Calc(select=[a]) +- MultipleInput(readOrder=[0,1,0,1], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = g)], select=[a, g], build=[right])\n:- Union(all=[true], union=[a])\n: :- Calc(select=[a])\n: : +- NestedLoopJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left])\n: : :- [#3] Exchange(distribution=[broadcast])\n: : +- [#4] Calc(select=[d])\n: +- [#2] Calc(select=[a])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[g]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, z, source: [TestTableSource(g, h, i, nz)]]], fields=[g, h, i, nz])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, z]], fields=[g, h, i, nz])(reuse_id=[1]) :- Calc(select=[a]) : +- NestedLoopJoin(joinType=[FullOuterJoin], where=[(a > cnt)], select=[a, cnt], build=[right], singleRowJoin=[true]) : :- Exchange(distribution=[single]) @@ -163,9 +163,9 @@ Calc(select=[a]) : +- Reused(reference_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) ]]> @@ -190,15 +190,15 @@ LogicalProject(a=[$0]) : :- LogicalProject(a=[$0]) : : +- LogicalProject(a=[$0]) : : +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) : +- LogicalProject(a=[$0]) : +- LogicalJoin(condition=[>($0, $3)], joinType=[full]) : :- LogicalTableScan(table=[[default_catalog, default_database, t]]) : +- LogicalAggregate(group=[{}], cnt=[COUNT()]) : +- LogicalProject($f0=[0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(g, h, i, nz)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(g, h, i, nz)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -207,7 +207,7 @@ Calc(select=[a]) +- MultipleInput(readOrder=[0,1,0,1], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = g)], select=[a, g], build=[right])\n:- Union(all=[true], union=[a])\n: :- Calc(select=[a])\n: : +- NestedLoopJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left])\n: : :- [#3] Exchange(distribution=[broadcast])\n: : +- [#4] Calc(select=[d])\n: +- [#2] Calc(select=[a])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[g]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, z, source: [TestTableSource(g, h, i, nz)]]], fields=[g, h, i, nz])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, z]], fields=[g, h, i, nz])(reuse_id=[1]) :- Calc(select=[a]) : +- NestedLoopJoin(joinType=[FullOuterJoin], where=[(a > cnt)], select=[a, cnt], build=[right], singleRowJoin=[true]) : :- Exchange(distribution=[single]) @@ -220,9 +220,9 @@ Calc(select=[a]) : +- Reused(reference_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) ]]> @@ -247,15 +247,15 @@ LogicalProject(a=[$0]) : :- LogicalProject(a=[$0]) : : +- LogicalProject(a=[$0]) : : +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) : +- LogicalProject(a=[$0]) : +- LogicalJoin(condition=[>($0, $3)], joinType=[full]) : :- LogicalTableScan(table=[[default_catalog, default_database, t]]) : +- LogicalAggregate(group=[{}], cnt=[COUNT()]) : +- LogicalProject($f0=[0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(g, h, i, nz)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(g, h, i, nz)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -264,7 +264,7 @@ Calc(select=[a]) +- MultipleInput(readOrder=[0,1,0,1], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = g)], select=[a, g], build=[right])\n:- Union(all=[true], union=[a])\n: :- Calc(select=[a])\n: : +- NestedLoopJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left])\n: : :- [#3] Exchange(distribution=[broadcast])\n: : +- [#4] Calc(select=[d])\n: +- [#2] Calc(select=[a])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[g]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, z, source: [TestTableSource(g, h, i, nz)]]], fields=[g, h, i, nz])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, z]], fields=[g, h, i, nz])(reuse_id=[1]) :- Calc(select=[a]) : +- NestedLoopJoin(joinType=[FullOuterJoin], where=[(a > cnt)], select=[a, cnt], build=[right], singleRowJoin=[true]) : :- Exchange(distribution=[single]) @@ -277,9 +277,9 @@ Calc(select=[a]) : +- Reused(reference_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) ]]> @@ -298,10 +298,10 @@ LogicalProject(a=[$0], d=[$1], e=[$2], f=[$3], ny=[$4]) :- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0]) : : +- LogicalProject(a=[$0]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(a=[$0]) : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -309,11 +309,11 @@ LogicalProject(a=[$0], d=[$1], e=[$2], f=[$3], ny=[$4]) NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = d)], select=[a, d, e, f, ny], build=[right]) :- Union(all=[true], union=[a]) : :- Calc(select=[a]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) : +- Calc(select=[a]) : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) ]]> @@ -332,10 +332,10 @@ LogicalProject(a=[$0], d=[$1], e=[$2], f=[$3], ny=[$4]) :- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0]) : : +- LogicalProject(a=[$0]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(a=[$0]) : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -343,11 +343,11 @@ LogicalProject(a=[$0], d=[$1], e=[$2], f=[$3], ny=[$4]) NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = d)], select=[a, d, e, f, ny], build=[right]) :- Union(all=[true], union=[a]) : :- Calc(select=[a]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) : +- Calc(select=[a]) : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) ]]> @@ -366,10 +366,10 @@ LogicalProject(a=[$0], d=[$1], e=[$2], f=[$3], ny=[$4]) :- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0]) : : +- LogicalProject(a=[$0]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(a=[$0]) : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -377,11 +377,11 @@ LogicalProject(a=[$0], d=[$1], e=[$2], f=[$3], ny=[$4]) NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = d)], select=[a, d, e, f, ny], build=[right]) :- Union(all=[true], union=[a]) : :- Calc(select=[a]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) : +- Calc(select=[a]) : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) ]]> @@ -401,11 +401,11 @@ LogicalProject(a=[$0], d=[$1]) +- LogicalJoin(condition=[=($0, $1)], joinType=[inner]) :- LogicalProject(a=[$0]) : +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(d=[$0]) +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -414,10 +414,10 @@ LogicalProject(a=[$0], d=[$1]) MultipleInput(readOrder=[0,1,1,2], members=[\nHashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left])\n:- Calc(select=[a])\n: +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left])\n: :- [#1] Exchange(distribution=[hash[a]])\n: +- [#2] Exchange(distribution=[hash[d]])\n+- Calc(select=[d])\n +- HashJoin(joinType=[InnerJoin], where=[(d = a)], select=[d, a], build=[left])\n :- [#2] Exchange(distribution=[hash[d]])\n +- [#4] Exchange(distribution=[hash[a]])\n]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) :- Exchange(distribution=[hash[d]])(reuse_id=[1]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) :- Reused(reference_id=[1]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a]) @@ -441,11 +441,11 @@ LogicalProject(a=[$0], d=[$1]) +- LogicalJoin(condition=[=($0, $1)], joinType=[inner]) :- LogicalProject(a=[$0]) : +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(d=[$0]) +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -454,10 +454,10 @@ LogicalProject(a=[$0], d=[$1]) MultipleInput(readOrder=[0,1,1,2], members=[\nHashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left])\n:- Calc(select=[a])\n: +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left])\n: :- [#1] Exchange(distribution=[hash[a]])\n: +- [#2] Exchange(distribution=[hash[d]])\n+- Calc(select=[d])\n +- HashJoin(joinType=[InnerJoin], where=[(d = a)], select=[d, a], build=[left])\n :- [#2] Exchange(distribution=[hash[d]])\n +- [#4] Exchange(distribution=[hash[a]])\n]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) :- Exchange(distribution=[hash[d]])(reuse_id=[1]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) :- Reused(reference_id=[1]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a]) @@ -481,11 +481,11 @@ LogicalProject(a=[$0], d=[$1]) +- LogicalJoin(condition=[=($0, $1)], joinType=[inner]) :- LogicalProject(a=[$0]) : +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(d=[$0]) +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -494,10 +494,10 @@ LogicalProject(a=[$0], d=[$1]) MultipleInput(readOrder=[0,1,1,2], members=[\nHashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left])\n:- Calc(select=[a])\n: +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left])\n: :- [#1] Exchange(distribution=[hash[a]])\n: +- [#2] Exchange(distribution=[hash[d]])\n+- Calc(select=[d])\n +- HashJoin(joinType=[InnerJoin], where=[(d = a)], select=[d, a], build=[left])\n :- [#2] Exchange(distribution=[hash[d]])\n +- [#4] Exchange(distribution=[hash[a]])\n]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) :- Exchange(distribution=[hash[d]])(reuse_id=[1]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) :- Reused(reference_id=[1]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a]) @@ -523,10 +523,10 @@ LogicalUnion(all=[true]) : +- LogicalProject(a=[$0], EXPR$1=[+($0, 1)]) : +- LogicalProject(a=[$0]) : +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) -: :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) -: +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) +: :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -536,10 +536,10 @@ Union(all=[true], union=[a, EXPR$1]) : +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left]) : :- Exchange(distribution=[hash[a]]) : : +- Calc(select=[a]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx])(reuse_id=[1]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx])(reuse_id=[1]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) +- Calc(select=[a, b]) +- Reused(reference_id=[1]) ]]> @@ -563,10 +563,10 @@ LogicalUnion(all=[true]) : +- LogicalProject(a=[$0], EXPR$1=[+($0, 1)]) : +- LogicalProject(a=[$0]) : +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) -: :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) -: +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) +: :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -576,10 +576,10 @@ Union(all=[true], union=[a, EXPR$1]) : +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left]) : :- Exchange(distribution=[hash[a]]) : : +- Calc(select=[a]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx])(reuse_id=[1]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx])(reuse_id=[1]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) +- Calc(select=[a, b]) +- Reused(reference_id=[1]) ]]> @@ -603,10 +603,10 @@ LogicalUnion(all=[true]) : +- LogicalProject(a=[$0], EXPR$1=[+($0, 1)]) : +- LogicalProject(a=[$0]) : +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) -: :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) -: +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) +: :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -616,10 +616,10 @@ Union(all=[true], union=[a, EXPR$1]) : +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left]) : :- Exchange(distribution=[hash[a]]) : : +- Calc(select=[a]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx])(reuse_id=[1]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx])(reuse_id=[1]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) +- Calc(select=[a, b]) +- Reused(reference_id=[1]) ]]> @@ -639,12 +639,12 @@ SELECT x.a, x.b, T1.* FROM x LEFT JOIN T1 ON x.a = T1.k WHERE x.a > 0 AND T1.v = LogicalProject(a=[$0], b=[$1], a0=[$4], b0=[$5], c=[$6], nx=[$7], k=[$8], v=[$9]) +- LogicalFilter(condition=[AND(>($0, 0), =($9, 0))]) +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], nx=[$3], k=[$4], v=[+($1, $5)]) +- LogicalFilter(condition=[>($4, 0)]) +- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -653,7 +653,7 @@ Calc(select=[a, b, a0, b0, c, nx, k, CAST(0 AS BIGINT) AS v]) +- MultipleInput(readOrder=[2,1,0], members=[\nHashJoin(joinType=[InnerJoin], where=[(a = k)], select=[a, b, a0, b0, c, nx, k], build=[right])\n:- [#1] Exchange(distribution=[hash[a]], shuffle_mode=[BATCH])\n+- Calc(select=[a, b, c, nx, a0 AS k])\n +- HashJoin(joinType=[InnerJoin], where=[((a = a0) AND ((b + b0) = 0))], select=[a, b, c, nx, a0, b0], build=[right])\n :- [#2] Exchange(distribution=[hash[a]])\n +- [#3] Exchange(distribution=[hash[a]])\n]) :- Exchange(distribution=[hash[a]], shuffle_mode=[BATCH]) : +- Calc(select=[a, b], where=[(a > 0)])(reuse_id=[2]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx])(reuse_id=[1]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c, nx], where=[(a > 0)]) : +- Reused(reference_id=[1]) @@ -676,12 +676,12 @@ SELECT x.a, x.b, T1.* FROM x LEFT JOIN T1 ON x.a = T1.k WHERE x.a > 0 AND T1.v = LogicalProject(a=[$0], b=[$1], a0=[$4], b0=[$5], c=[$6], nx=[$7], k=[$8], v=[$9]) +- LogicalFilter(condition=[AND(>($0, 0), =($9, 0))]) +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], nx=[$3], k=[$4], v=[+($1, $5)]) +- LogicalFilter(condition=[>($4, 0)]) +- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -690,7 +690,7 @@ Calc(select=[a, b, a0, b0, c, nx, k, CAST(0 AS BIGINT) AS v]) +- MultipleInput(readOrder=[2,1,0], members=[\nHashJoin(joinType=[InnerJoin], where=[(a = k)], select=[a, b, a0, b0, c, nx, k], build=[right])\n:- [#1] Exchange(distribution=[hash[a]], shuffle_mode=[BATCH])\n+- Calc(select=[a, b, c, nx, a0 AS k])\n +- HashJoin(joinType=[InnerJoin], where=[((a = a0) AND ((b + b0) = 0))], select=[a, b, c, nx, a0, b0], build=[right])\n :- [#2] Exchange(distribution=[hash[a]])\n +- [#3] Exchange(distribution=[hash[a]])\n]) :- Exchange(distribution=[hash[a]], shuffle_mode=[BATCH]) : +- Calc(select=[a, b], where=[(a > 0)])(reuse_id=[2]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx])(reuse_id=[1]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c, nx], where=[(a > 0)]) : +- Reused(reference_id=[1]) @@ -713,12 +713,12 @@ SELECT x.a, x.b, T1.* FROM x LEFT JOIN T1 ON x.a = T1.k WHERE x.a > 0 AND T1.v = LogicalProject(a=[$0], b=[$1], a0=[$4], b0=[$5], c=[$6], nx=[$7], k=[$8], v=[$9]) +- LogicalFilter(condition=[AND(>($0, 0), =($9, 0))]) +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], nx=[$3], k=[$4], v=[+($1, $5)]) +- LogicalFilter(condition=[>($4, 0)]) +- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -727,7 +727,7 @@ Calc(select=[a, b, a0, b0, c, nx, k, CAST(0 AS BIGINT) AS v]) +- MultipleInput(readOrder=[2,1,0], members=[\nHashJoin(joinType=[InnerJoin], where=[(a = k)], select=[a, b, a0, b0, c, nx, k], build=[right])\n:- [#1] Exchange(distribution=[hash[a]], shuffle_mode=[BATCH])\n+- Calc(select=[a, b, c, nx, a0 AS k])\n +- HashJoin(joinType=[InnerJoin], where=[((a = a0) AND ((b + b0) = 0))], select=[a, b, c, nx, a0, b0], build=[right])\n :- [#2] Exchange(distribution=[hash[a]], shuffle_mode=[BATCH])\n +- [#3] Exchange(distribution=[hash[a]])\n]) :- Exchange(distribution=[hash[a]], shuffle_mode=[BATCH]) : +- Calc(select=[a, b], where=[(a > 0)])(reuse_id=[2]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx])(reuse_id=[1]) :- Exchange(distribution=[hash[a]], shuffle_mode=[BATCH]) : +- Calc(select=[a, b, c, nx], where=[(a > 0)]) : +- Reused(reference_id=[1]) @@ -752,7 +752,7 @@ LogicalProject(a=[$0], d=[$1], e=[$2], f=[$3], ny=[$4], a0=[$5], b=[$6], c=[$7]) +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) :- LogicalJoin(condition=[=($0, $1)], joinType=[left]) : :- LogicalTableScan(table=[[default_catalog, default_database, chainable]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -764,7 +764,7 @@ MultipleInput(readOrder=[0,0,1], members=[\nNestedLoopJoin(joinType=[LeftOuterJo : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[d, e, f, ny], where=[(d > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) +- BoundedStreamScan(table=[[default_catalog, default_database, chainable]], fields=[a]) ]]> @@ -785,7 +785,7 @@ LogicalProject(a=[$0], d=[$1], e=[$2], f=[$3], ny=[$4], a0=[$5], b=[$6], c=[$7]) +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) :- LogicalJoin(condition=[=($0, $1)], joinType=[left]) : :- LogicalTableScan(table=[[default_catalog, default_database, chainable]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -797,7 +797,7 @@ MultipleInput(readOrder=[0,0,1], members=[\nNestedLoopJoin(joinType=[LeftOuterJo : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[d, e, f, ny], where=[(d > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) +- BoundedStreamScan(table=[[default_catalog, default_database, chainable]], fields=[a]) ]]> @@ -818,7 +818,7 @@ LogicalProject(a=[$0], d=[$1], e=[$2], f=[$3], ny=[$4], a0=[$5], b=[$6], c=[$7]) +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) :- LogicalJoin(condition=[=($0, $1)], joinType=[left]) : :- LogicalTableScan(table=[[default_catalog, default_database, chainable]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -830,7 +830,7 @@ MultipleInput(readOrder=[0,0,1], members=[\nNestedLoopJoin(joinType=[LeftOuterJo : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[d, e, f, ny], where=[(d > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) +- BoundedStreamScan(table=[[default_catalog, default_database, chainable]], fields=[a]) ]]> @@ -853,14 +853,14 @@ LogicalProject(a=[$0], d=[$1], e=[$2], f=[$3], ny=[$4]) : : +- LogicalTableScan(table=[[default_catalog, default_database, chainable]]) : +- LogicalProject(a=[$0]) : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -971,10 +971,10 @@ HashJoin(joinType=[LeftOuterJoin], where=[(a = d)], select=[a, cnt, d, sm], buil : +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left])(reuse_id=[1]) : :- Exchange(distribution=[hash[a]]) : : +- Calc(select=[a]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) +- Exchange(distribution=[hash[d]]) +- HashAggregate(isMerge=[false], groupBy=[d], select=[d, SUM(a) AS sm]) +- Exchange(distribution=[keep_input_as_is[hash[d]]]) @@ -1001,14 +1001,14 @@ LogicalProject(a=[$0], cnt=[$1], d=[$2], sm=[$3]) : +- LogicalProject(a=[$0]) : +- LogicalProject(a=[$0], d=[$4]) : +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalAggregate(group=[{0}], sm=[SUM($1)]) +- LogicalProject(d=[$1], a=[$0]) +- LogicalProject(a=[$0], d=[$4]) +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1022,10 +1022,10 @@ HashJoin(joinType=[LeftOuterJoin], where=[(a = d)], select=[a, cnt, d, sm], buil : +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left])(reuse_id=[1]) : :- Exchange(distribution=[hash[a]]) : : +- Calc(select=[a]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) +- Exchange(distribution=[hash[d]]) +- HashAggregate(isMerge=[false], groupBy=[d], select=[d, SUM(a) AS sm]) +- Exchange(distribution=[keep_input_as_is[hash[d]]]) @@ -1052,14 +1052,14 @@ LogicalProject(a=[$0], cnt=[$1], d=[$2], sm=[$3]) : +- LogicalProject(a=[$0]) : +- LogicalProject(a=[$0], d=[$4]) : +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalAggregate(group=[{0}], sm=[SUM($1)]) +- LogicalProject(d=[$1], a=[$0]) +- LogicalProject(a=[$0], d=[$4]) +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1073,10 +1073,10 @@ HashJoin(joinType=[LeftOuterJoin], where=[(a = d)], select=[a, cnt, d, sm], buil : +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left])(reuse_id=[1]) : :- Exchange(distribution=[hash[a]]) : : +- Calc(select=[a]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) +- Exchange(distribution=[hash[d]]) +- HashAggregate(isMerge=[false], groupBy=[d], select=[d, SUM(a) AS sm]) +- Exchange(distribution=[keep_input_as_is[hash[d]]]) @@ -1093,7 +1093,7 @@ HashJoin(joinType=[LeftOuterJoin], where=[(a = d)], select=[a, cnt, d, sm], buil LogicalProject(a=[$0], a0=[$1], b=[$2], c=[$3], nx=[$4]) +- LogicalJoin(condition=[=($0, $1)], joinType=[left]) :- LogicalTableScan(table=[[default_catalog, default_database, chainable]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1101,7 +1101,7 @@ LogicalProject(a=[$0], a0=[$1], b=[$2], c=[$3], nx=[$4]) MultipleInput(readOrder=[1,0], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = a0)], select=[a, a0, b, c, nx], build=[right])\n:- [#1] BoundedStreamScan(table=[[default_catalog, default_database, chainable]], fields=[a])\n+- [#2] Exchange(distribution=[broadcast])\n]) :- BoundedStreamScan(table=[[default_catalog, default_database, chainable]], fields=[a]) +- Exchange(distribution=[broadcast]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) + +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) ]]> @@ -1114,7 +1114,7 @@ MultipleInput(readOrder=[1,0], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin LogicalProject(a=[$0], a0=[$1], b=[$2], c=[$3], nx=[$4]) +- LogicalJoin(condition=[=($0, $1)], joinType=[left]) :- LogicalTableScan(table=[[default_catalog, default_database, chainable]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1122,7 +1122,7 @@ LogicalProject(a=[$0], a0=[$1], b=[$2], c=[$3], nx=[$4]) MultipleInput(readOrder=[1,0], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = a0)], select=[a, a0, b, c, nx], build=[right])\n:- [#1] BoundedStreamScan(table=[[default_catalog, default_database, chainable]], fields=[a])\n+- [#2] Exchange(distribution=[broadcast])\n]) :- BoundedStreamScan(table=[[default_catalog, default_database, chainable]], fields=[a]) +- Exchange(distribution=[broadcast]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) + +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) ]]> @@ -1135,7 +1135,7 @@ MultipleInput(readOrder=[1,0], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin LogicalProject(a=[$0], a0=[$1], b=[$2], c=[$3], nx=[$4]) +- LogicalJoin(condition=[=($0, $1)], joinType=[left]) :- LogicalTableScan(table=[[default_catalog, default_database, chainable]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1143,7 +1143,7 @@ LogicalProject(a=[$0], a0=[$1], b=[$2], c=[$3], nx=[$4]) MultipleInput(readOrder=[1,0], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = a0)], select=[a, a0, b, c, nx], build=[right])\n:- [#1] BoundedStreamScan(table=[[default_catalog, default_database, chainable]], fields=[a])\n+- [#2] Exchange(distribution=[broadcast])\n]) :- BoundedStreamScan(table=[[default_catalog, default_database, chainable]], fields=[a]) +- Exchange(distribution=[broadcast]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) + +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) ]]> @@ -1167,12 +1167,12 @@ LogicalUnion(all=[true]) : +- LogicalProject(a=[$0]) : +- LogicalJoin(condition=[=($0, $1)], joinType=[left]) : :- LogicalTableScan(table=[[default_catalog, default_database, chainable]]) -: +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0]) +- LogicalProject(a=[$0]) +- LogicalJoin(condition=[=($0, $1)], joinType=[left]) :- LogicalTableScan(table=[[default_catalog, default_database, chainable]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1181,11 +1181,11 @@ MultipleInput(readOrder=[1,0,1,0], members=[\nUnion(all=[true], union=[a])\n:- C :- BoundedStreamScan(table=[[default_catalog, default_database, chainable]], fields=[a])(reuse_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) :- Reused(reference_id=[1]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) ]]> @@ -1209,12 +1209,12 @@ LogicalUnion(all=[true]) : +- LogicalProject(a=[$0]) : +- LogicalJoin(condition=[=($0, $1)], joinType=[left]) : :- LogicalTableScan(table=[[default_catalog, default_database, chainable]]) -: +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0]) +- LogicalProject(a=[$0]) +- LogicalJoin(condition=[=($0, $1)], joinType=[left]) :- LogicalTableScan(table=[[default_catalog, default_database, chainable]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1223,11 +1223,11 @@ MultipleInput(readOrder=[1,0,1,0], members=[\nUnion(all=[true], union=[a])\n:- C :- BoundedStreamScan(table=[[default_catalog, default_database, chainable]], fields=[a])(reuse_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) :- Reused(reference_id=[1]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) ]]> @@ -1251,12 +1251,12 @@ LogicalUnion(all=[true]) : +- LogicalProject(a=[$0]) : +- LogicalJoin(condition=[=($0, $1)], joinType=[left]) : :- LogicalTableScan(table=[[default_catalog, default_database, chainable]]) -: +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0]) +- LogicalProject(a=[$0]) +- LogicalJoin(condition=[=($0, $1)], joinType=[left]) :- LogicalTableScan(table=[[default_catalog, default_database, chainable]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1265,11 +1265,11 @@ MultipleInput(readOrder=[1,0,1,0], members=[\nUnion(all=[true], union=[a])\n:- C :- BoundedStreamScan(table=[[default_catalog, default_database, chainable]], fields=[a])(reuse_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) :- Reused(reference_id=[1]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) ]]> @@ -1311,11 +1311,11 @@ LogicalUnion(all=[true]) : : : :- LogicalProject(a=[$0], ny=[$7], nz=[$11]) : : : : +- LogicalJoin(condition=[=($0, $11)], joinType=[left]) : : : : :- LogicalJoin(condition=[=($0, $7)], joinType=[left]) -: : : : : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) -: : : : : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) -: : : : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(g, h, i, nz)]]]) +: : : : : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +: : : : : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +: : : : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) : : : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) -: : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) +: : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) +- LogicalProject(e=[$5], sd=[$1], sy=[$2], sz=[$3]) +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) @@ -1327,12 +1327,12 @@ LogicalUnion(all=[true]) : : :- LogicalProject(a=[$0], ny=[$7], nz=[$11]) : : : +- LogicalJoin(condition=[=($0, $11)], joinType=[left]) : : : :- LogicalJoin(condition=[=($0, $7)], joinType=[left]) - : : : : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : : : : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(g, h, i, nz)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : : : : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : : : : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1343,23 +1343,23 @@ MultipleInput(readOrder=[0,0,1,1], members=[\nUnion(all=[true], union=[b, sd, sy : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c])(reuse_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[e]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) :- Exchange(distribution=[hash[b]]) : +- LocalHashAggregate(groupBy=[b], select=[b, Partial_SUM(d) AS sum$0, Partial_SUM(ny) AS sum$1, Partial_SUM(nz) AS sum$2]) : +- Calc(select=[b, d, ny, nz]) : +- MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, ny, nz, b, d], build=[right])\n:- Calc(select=[a, ny, nz, b])\n: +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = a0)], select=[a, ny, nz, a0, b], build=[right])\n: :- [#2] MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = nz)], select=[a, ny, nz], build=[right])\n:- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = ny)], select=[a, ny], build=[right])\n: :- [#2] Calc(select=[a])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) : :- Exchange(distribution=[broadcast])(reuse_id=[4]) : : +- Calc(select=[d]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: : +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) : :- MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = nz)], select=[a, ny, nz], build=[right])\n:- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = ny)], select=[a, ny], build=[right])\n: :- [#2] Calc(select=[a])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n])(reuse_id=[3]) : : :- Exchange(distribution=[broadcast]) : : : +- Calc(select=[nz]) -: : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, z, source: [TestTableSource(g, h, i, nz)]]], fields=[g, h, i, nz]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, z]], fields=[g, h, i, nz]) : : :- Calc(select=[a]) -: : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) : : +- Exchange(distribution=[broadcast]) : : +- Calc(select=[ny]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: : +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) : +- Exchange(distribution=[broadcast])(reuse_id=[2]) : +- Calc(select=[a, b]) : +- Reused(reference_id=[1]) @@ -1411,11 +1411,11 @@ LogicalUnion(all=[true]) : : : :- LogicalProject(a=[$0], ny=[$7], nz=[$11]) : : : : +- LogicalJoin(condition=[=($0, $11)], joinType=[left]) : : : : :- LogicalJoin(condition=[=($0, $7)], joinType=[left]) -: : : : : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) -: : : : : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) -: : : : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(g, h, i, nz)]]]) +: : : : : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +: : : : : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +: : : : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) : : : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) -: : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) +: : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) +- LogicalProject(e=[$5], sd=[$1], sy=[$2], sz=[$3]) +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) @@ -1427,12 +1427,12 @@ LogicalUnion(all=[true]) : : :- LogicalProject(a=[$0], ny=[$7], nz=[$11]) : : : +- LogicalJoin(condition=[=($0, $11)], joinType=[left]) : : : :- LogicalJoin(condition=[=($0, $7)], joinType=[left]) - : : : : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : : : : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(g, h, i, nz)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : : : : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : : : : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1443,23 +1443,23 @@ MultipleInput(readOrder=[0,0,1,1], members=[\nUnion(all=[true], union=[b, sd, sy : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c])(reuse_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[e]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) :- Exchange(distribution=[hash[b]]) : +- LocalHashAggregate(groupBy=[b], select=[b, Partial_SUM(d) AS sum$0, Partial_SUM(ny) AS sum$1, Partial_SUM(nz) AS sum$2]) : +- Calc(select=[b, d, ny, nz]) : +- MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, ny, nz, b, d], build=[right])\n:- Calc(select=[a, ny, nz, b])\n: +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = a0)], select=[a, ny, nz, a0, b], build=[right])\n: :- [#2] MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = nz)], select=[a, ny, nz], build=[right])\n:- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = ny)], select=[a, ny], build=[right])\n: :- [#2] Calc(select=[a])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) : :- Exchange(distribution=[broadcast])(reuse_id=[4]) : : +- Calc(select=[d]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: : +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) : :- MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = nz)], select=[a, ny, nz], build=[right])\n:- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = ny)], select=[a, ny], build=[right])\n: :- [#2] Calc(select=[a])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n])(reuse_id=[3]) : : :- Exchange(distribution=[broadcast]) : : : +- Calc(select=[nz]) -: : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, z, source: [TestTableSource(g, h, i, nz)]]], fields=[g, h, i, nz]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, z]], fields=[g, h, i, nz]) : : :- Calc(select=[a]) -: : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) : : +- Exchange(distribution=[broadcast]) : : +- Calc(select=[ny]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: : +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) : +- Exchange(distribution=[broadcast])(reuse_id=[2]) : +- Calc(select=[a, b]) : +- Reused(reference_id=[1]) @@ -1511,11 +1511,11 @@ LogicalUnion(all=[true]) : : : :- LogicalProject(a=[$0], ny=[$7], nz=[$11]) : : : : +- LogicalJoin(condition=[=($0, $11)], joinType=[left]) : : : : :- LogicalJoin(condition=[=($0, $7)], joinType=[left]) -: : : : : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) -: : : : : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) -: : : : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(g, h, i, nz)]]]) +: : : : : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +: : : : : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +: : : : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) : : : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) -: : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) +: : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) +- LogicalProject(e=[$5], sd=[$1], sy=[$2], sz=[$3]) +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) @@ -1527,12 +1527,12 @@ LogicalUnion(all=[true]) : : :- LogicalProject(a=[$0], ny=[$7], nz=[$11]) : : : +- LogicalJoin(condition=[=($0, $11)], joinType=[left]) : : : :- LogicalJoin(condition=[=($0, $7)], joinType=[left]) - : : : : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : : : : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(g, h, i, nz)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : : : : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : : : : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1543,23 +1543,23 @@ MultipleInput(readOrder=[0,0,1,1], members=[\nUnion(all=[true], union=[b, sd, sy : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c])(reuse_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[e]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) :- Exchange(distribution=[hash[b]]) : +- LocalHashAggregate(groupBy=[b], select=[b, Partial_SUM(d) AS sum$0, Partial_SUM(ny) AS sum$1, Partial_SUM(nz) AS sum$2]) : +- Calc(select=[b, d, ny, nz]) : +- MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, ny, nz, b, d], build=[right])\n:- Calc(select=[a, ny, nz, b])\n: +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = a0)], select=[a, ny, nz, a0, b], build=[right])\n: :- [#2] MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = nz)], select=[a, ny, nz], build=[right])\n:- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = ny)], select=[a, ny], build=[right])\n: :- [#2] Calc(select=[a])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) : :- Exchange(distribution=[broadcast])(reuse_id=[4]) : : +- Calc(select=[d]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: : +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) : :- MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = nz)], select=[a, ny, nz], build=[right])\n:- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = ny)], select=[a, ny], build=[right])\n: :- [#2] Calc(select=[a])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n])(reuse_id=[3]) : : :- Exchange(distribution=[broadcast]) : : : +- Calc(select=[nz]) -: : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, z, source: [TestTableSource(g, h, i, nz)]]], fields=[g, h, i, nz]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, z]], fields=[g, h, i, nz]) : : :- Calc(select=[a]) -: : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) : : +- Exchange(distribution=[broadcast]) : : +- Calc(select=[ny]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: : +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) : +- Exchange(distribution=[broadcast])(reuse_id=[2]) : +- Calc(select=[a, b]) : +- Reused(reference_id=[1]) @@ -1586,8 +1586,8 @@ SELECT * FROM x LogicalProject(a=[$0], b=[$1], c=[$2], nx=[$3], d=[$4], e=[$5], f=[$6], ny=[$7], a0=[$8], b0=[$9], c0=[$10]) +- LogicalJoin(condition=[=($0, $8)], joinType=[inner]) :- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -1597,9 +1597,9 @@ MultipleInput(members=[\nSortMergeJoin(joinType=[InnerJoin], where=[(a = a0)], s :- Exchange(distribution=[hash[a]]) : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) :- Exchange(distribution=[hash[a]]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) +- Exchange(distribution=[hash[d]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) ]]> @@ -1616,8 +1616,8 @@ SELECT * FROM x LogicalProject(a=[$0], b=[$1], c=[$2], nx=[$3], d=[$4], e=[$5], f=[$6], ny=[$7], a0=[$8], b0=[$9], c0=[$10]) +- LogicalJoin(condition=[=($0, $8)], joinType=[inner]) :- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -1627,9 +1627,9 @@ MultipleInput(members=[\nSortMergeJoin(joinType=[InnerJoin], where=[(a = a0)], s :- Exchange(distribution=[hash[a]]) : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) :- Exchange(distribution=[hash[a]]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) +- Exchange(distribution=[hash[d]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) ]]> @@ -1646,8 +1646,8 @@ SELECT * FROM x LogicalProject(a=[$0], b=[$1], c=[$2], nx=[$3], d=[$4], e=[$5], f=[$6], ny=[$7], a0=[$8], b0=[$9], c0=[$10]) +- LogicalJoin(condition=[=($0, $8)], joinType=[inner]) :- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -1657,9 +1657,9 @@ MultipleInput(members=[\nSortMergeJoin(joinType=[InnerJoin], where=[(a = a0)], s :- Exchange(distribution=[hash[a]]) : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) :- Exchange(distribution=[hash[a]]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) +- Exchange(distribution=[hash[d]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) ]]> @@ -1685,10 +1685,10 @@ LogicalProject(a=[$0], b=[$1], a0=[$2], b0=[$3], c=[$4]) : : +- LogicalProject(a=[$0], b=[$1]) : : +- LogicalProject(a=[$4], b=[$0]) : : +- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, y]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -1698,9 +1698,9 @@ MultipleInput(readOrder=[0,1,2,1], members=[\nNestedLoopJoin(joinType=[LeftOuter :- Exchange(distribution=[broadcast]) : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) :- Calc(select=[a, b]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx])(reuse_id=[1]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx])(reuse_id=[1]) :- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) +- Exchange(distribution=[broadcast]) +- Calc(select=[a]) +- Reused(reference_id=[1]) @@ -1729,10 +1729,10 @@ LogicalProject(a=[$0], b=[$1], a0=[$2], b0=[$3], c=[$4]) : : +- LogicalProject(a=[$0], b=[$1]) : : +- LogicalProject(a=[$4], b=[$0]) : : +- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, y]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -1742,9 +1742,9 @@ MultipleInput(readOrder=[0,1,2,1], members=[\nNestedLoopJoin(joinType=[LeftOuter :- Exchange(distribution=[broadcast]) : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) :- Calc(select=[a, b]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx])(reuse_id=[1]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx])(reuse_id=[1]) :- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) +- Exchange(distribution=[broadcast]) +- Calc(select=[a]) +- Reused(reference_id=[1]) @@ -1773,10 +1773,10 @@ LogicalProject(a=[$0], b=[$1], a0=[$2], b0=[$3], c=[$4]) : : +- LogicalProject(a=[$0], b=[$1]) : : +- LogicalProject(a=[$4], b=[$0]) : : +- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, y]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -1786,9 +1786,9 @@ MultipleInput(readOrder=[0,1,2,1], members=[\nNestedLoopJoin(joinType=[LeftOuter :- Exchange(distribution=[broadcast]) : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) :- Calc(select=[a, b]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx])(reuse_id=[1]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx])(reuse_id=[1]) :- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) +- Exchange(distribution=[broadcast]) +- Calc(select=[a]) +- Reused(reference_id=[1]) @@ -1818,12 +1818,12 @@ LogicalProject(a=[$0], b=[$1], a0=[$2], b0=[$3], c=[$4]) : : +- LogicalProject(a=[$0], b=[$1]) : : +- LogicalProject(a=[$4], b=[$0]) : : +- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, y]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(a=[$1], b=[$0]) : +- LogicalAggregate(group=[{0}], a=[COUNT($1)]) : +- LogicalProject(b=[$1], a=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], a0=[CAST($0):BIGINT]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -1836,10 +1836,10 @@ Calc(select=[a, b, a0, b0, c]) : +- Calc(select=[a, b, c, CAST(a AS BIGINT) AS a0]) : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) :- Calc(select=[d]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + : +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx])(reuse_id=[1]) +- Exchange(distribution=[hash[b]]) +- LocalHashAggregate(groupBy=[b], select=[b, Partial_COUNT(a) AS count$0]) +- Calc(select=[b, a]) @@ -1870,12 +1870,12 @@ LogicalProject(a=[$0], b=[$1], a0=[$2], b0=[$3], c=[$4]) : : +- LogicalProject(a=[$0], b=[$1]) : : +- LogicalProject(a=[$4], b=[$0]) : : +- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, y]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(a=[$1], b=[$0]) : +- LogicalAggregate(group=[{0}], a=[COUNT($1)]) : +- LogicalProject(b=[$1], a=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], a0=[CAST($0):BIGINT]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -1888,10 +1888,10 @@ Calc(select=[a, b, a0, b0, c]) : +- Calc(select=[a, b, c, CAST(a AS BIGINT) AS a0]) : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) :- Calc(select=[d]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + : +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx])(reuse_id=[1]) +- Exchange(distribution=[hash[b]]) +- LocalHashAggregate(groupBy=[b], select=[b, Partial_COUNT(a) AS count$0]) +- Calc(select=[b, a]) @@ -1922,12 +1922,12 @@ LogicalProject(a=[$0], b=[$1], a0=[$2], b0=[$3], c=[$4]) : : +- LogicalProject(a=[$0], b=[$1]) : : +- LogicalProject(a=[$4], b=[$0]) : : +- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, y]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(a=[$1], b=[$0]) : +- LogicalAggregate(group=[{0}], a=[COUNT($1)]) : +- LogicalProject(b=[$1], a=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], a0=[CAST($0):BIGINT]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -1940,10 +1940,10 @@ Calc(select=[a, b, a0, b0, c]) : +- Calc(select=[a, b, c, CAST(a AS BIGINT) AS a0]) : +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) :- Calc(select=[d]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + : +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx])(reuse_id=[1]) +- Exchange(distribution=[hash[b]]) +- LocalHashAggregate(groupBy=[b], select=[b, Partial_COUNT(a) AS count$0]) +- Calc(select=[b, a]) @@ -1973,14 +1973,14 @@ LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], a0=[CAST($0):BIGINT]) : : +- LogicalProject(a=[$0]) : : +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) -: : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) -: : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) +: : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +: : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) : +- LogicalProject(b=[$1]) : +- LogicalJoin(condition=[=($1, $5)], joinType=[inner]) -: :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) -: +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) +: :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1993,18 +1993,18 @@ Union(all=[true], union=[a, b]) : : +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left]) : : :- Exchange(distribution=[hash[a]]) : : : +- Calc(select=[a]) -: : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) : : +- Exchange(distribution=[hash[d]]) : : +- Calc(select=[d]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: : +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) : :- Exchange(distribution=[hash[b]]) : : +- Calc(select=[b]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) : +- Exchange(distribution=[hash[e]]) : +- Calc(select=[e]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) + +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) ]]> @@ -2030,14 +2030,14 @@ LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], a0=[CAST($0):BIGINT]) : : +- LogicalProject(a=[$0]) : : +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) -: : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) -: : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) +: : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +: : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) : +- LogicalProject(b=[$1]) : +- LogicalJoin(condition=[=($1, $5)], joinType=[inner]) -: :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) -: +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) +: :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -2050,18 +2050,18 @@ Union(all=[true], union=[a, b]) : : +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left]) : : :- Exchange(distribution=[hash[a]]) : : : +- Calc(select=[a]) -: : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) : : +- Exchange(distribution=[hash[d]]) : : +- Calc(select=[d]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: : +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) : :- Exchange(distribution=[hash[b]]) : : +- Calc(select=[b]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) : +- Exchange(distribution=[hash[e]]) : +- Calc(select=[e]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) + +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) ]]> @@ -2087,14 +2087,14 @@ LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], a0=[CAST($0):BIGINT]) : : +- LogicalProject(a=[$0]) : : +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) -: : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) -: : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) +: : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +: : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) : +- LogicalProject(b=[$1]) : +- LogicalJoin(condition=[=($1, $5)], joinType=[inner]) -: :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) -: +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) +: :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -2107,18 +2107,18 @@ Union(all=[true], union=[a, b]) : : +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left]) : : :- Exchange(distribution=[hash[a]]) : : : +- Calc(select=[a]) -: : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) : : +- Exchange(distribution=[hash[d]]) : : +- Calc(select=[d]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: : +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) : :- Exchange(distribution=[hash[b]]) : : +- Calc(select=[b]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) : +- Exchange(distribution=[hash[e]]) : +- Calc(select=[e]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) + +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) ]]> @@ -2146,24 +2146,24 @@ LogicalUnion(all=[true]) : : +- LogicalProject(cnt=[$1]) : : +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) : : +- LogicalProject(a=[$0]) -: : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) +: : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(cnt=[$0]) : +- LogicalProject(cnt=[$1]) : +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) : +- LogicalProject(d=[$0]) -: +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalUnion(all=[true]) :- LogicalProject(cnt=[$0]) : +- LogicalProject(cnt=[$0]) : +- LogicalProject(cnt=[$0]) : +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(cnt=[$0]) +- LogicalProject(cnt=[$1]) +- LogicalJoin(condition=[=($1, $5)], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -2175,13 +2175,13 @@ Union(all=[true], union=[cnt]) : : +- Exchange(distribution=[hash[a]]) : : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(*) AS count1$0]) : : +- Calc(select=[a])(reuse_id=[1]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) : +- Calc(select=[CAST(cnt AS BIGINT) AS cnt]) : +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count1$0) AS cnt]) : +- Exchange(distribution=[hash[d]]) : +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(*) AS count1$0]) : +- Calc(select=[d])(reuse_id=[2]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) +- MultipleInput(readOrder=[0,1,0,1], members=[\nUnion(all=[true], union=[cnt])\n:- Calc(select=[CAST(a AS BIGINT) AS cnt])\n: +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left])\n: :- [#1] Exchange(distribution=[hash[a]])\n: +- [#2] Exchange(distribution=[hash[d]])\n+- Calc(select=[b AS cnt])\n +- HashJoin(joinType=[InnerJoin], where=[(b = e)], select=[b, e], build=[left])\n :- [#3] Exchange(distribution=[hash[b]])\n +- [#4] Exchange(distribution=[hash[e]])\n]) :- Exchange(distribution=[hash[a]]) : +- Reused(reference_id=[1]) @@ -2189,10 +2189,10 @@ Union(all=[true], union=[cnt]) : +- Reused(reference_id=[2]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) ]]> @@ -2220,24 +2220,24 @@ LogicalUnion(all=[true]) : : +- LogicalProject(cnt=[$1]) : : +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) : : +- LogicalProject(a=[$0]) -: : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) +: : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(cnt=[$0]) : +- LogicalProject(cnt=[$1]) : +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) : +- LogicalProject(d=[$0]) -: +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalUnion(all=[true]) :- LogicalProject(cnt=[$0]) : +- LogicalProject(cnt=[$0]) : +- LogicalProject(cnt=[$0]) : +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(cnt=[$0]) +- LogicalProject(cnt=[$1]) +- LogicalJoin(condition=[=($1, $5)], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -2249,13 +2249,13 @@ Union(all=[true], union=[cnt]) : : +- Exchange(distribution=[hash[a]]) : : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(*) AS count1$0]) : : +- Calc(select=[a])(reuse_id=[1]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) : +- Calc(select=[CAST(cnt AS BIGINT) AS cnt]) : +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count1$0) AS cnt]) : +- Exchange(distribution=[hash[d]]) : +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(*) AS count1$0]) : +- Calc(select=[d])(reuse_id=[2]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) +- MultipleInput(readOrder=[0,1,0,1], members=[\nUnion(all=[true], union=[cnt])\n:- Calc(select=[CAST(a AS BIGINT) AS cnt])\n: +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left])\n: :- [#1] Exchange(distribution=[hash[a]])\n: +- [#2] Exchange(distribution=[hash[d]])\n+- Calc(select=[b AS cnt])\n +- HashJoin(joinType=[InnerJoin], where=[(b = e)], select=[b, e], build=[left])\n :- [#3] Exchange(distribution=[hash[b]])\n +- [#4] Exchange(distribution=[hash[e]])\n]) :- Exchange(distribution=[hash[a]]) : +- Reused(reference_id=[1]) @@ -2263,10 +2263,10 @@ Union(all=[true], union=[cnt]) : +- Reused(reference_id=[2]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) ]]> @@ -2294,24 +2294,24 @@ LogicalUnion(all=[true]) : : +- LogicalProject(cnt=[$1]) : : +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) : : +- LogicalProject(a=[$0]) -: : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) +: : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(cnt=[$0]) : +- LogicalProject(cnt=[$1]) : +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) : +- LogicalProject(d=[$0]) -: +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalUnion(all=[true]) :- LogicalProject(cnt=[$0]) : +- LogicalProject(cnt=[$0]) : +- LogicalProject(cnt=[$0]) : +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(cnt=[$0]) +- LogicalProject(cnt=[$1]) +- LogicalJoin(condition=[=($1, $5)], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -2323,13 +2323,13 @@ Union(all=[true], union=[cnt]) : : +- Exchange(distribution=[hash[a]]) : : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(*) AS count1$0]) : : +- Calc(select=[a])(reuse_id=[1]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) : +- Calc(select=[CAST(cnt AS BIGINT) AS cnt]) : +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count1$0) AS cnt]) : +- Exchange(distribution=[hash[d]]) : +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(*) AS count1$0]) : +- Calc(select=[d])(reuse_id=[2]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) +- MultipleInput(readOrder=[0,1,0,1], members=[\nUnion(all=[true], union=[cnt])\n:- Calc(select=[CAST(a AS BIGINT) AS cnt])\n: +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left])\n: :- [#1] Exchange(distribution=[hash[a]])\n: +- [#2] Exchange(distribution=[hash[d]])\n+- Calc(select=[b AS cnt])\n +- HashJoin(joinType=[InnerJoin], where=[(b = e)], select=[b, e], build=[left])\n :- [#3] Exchange(distribution=[hash[b]])\n +- [#4] Exchange(distribution=[hash[e]])\n]) :- Exchange(distribution=[hash[a]]) : +- Reused(reference_id=[1]) @@ -2337,10 +2337,10 @@ Union(all=[true], union=[cnt]) : +- Reused(reference_id=[2]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f, ny]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.xml index 0989e80d5c6a2..7c475b9629138 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.xml @@ -24,14 +24,14 @@ limitations under the License. @@ -43,7 +43,7 @@ Sink(table=[default_catalog.default_database.sink], fields=[a, b, c]) @@ -51,7 +51,7 @@ LogicalSink(table=[default_catalog.default_database.sinkShuffleBy], fields=[a, b Sink(table=[default_catalog.default_database.sinkShuffleBy], fields=[a, b, c]) +- Sort(orderBy=[b ASC, c ASC]) +- Exchange(distribution=[hash[b, c]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -63,7 +63,7 @@ Sink(table=[default_catalog.default_database.sinkShuffleBy], fields=[a, b, c]) @@ -71,7 +71,7 @@ LogicalSink(table=[default_catalog.default_database.sink], fields=[a, EXPR$1, c] Sink(table=[default_catalog.default_database.sink], fields=[a, EXPR$1, c]) +- Sort(orderBy=[c ASC]) +- Calc(select=[a, 1 AS EXPR$1, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -83,14 +83,14 @@ Sink(table=[default_catalog.default_database.sink], fields=[a, EXPR$1, c]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RankTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RankTest.xml index 1f15f4a1f2a1e..4e5aad99d8b13 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RankTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RankTest.xml @@ -58,7 +58,7 @@ WHERE rk < 10 LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[<($2, 10)]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, rk)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -71,7 +71,7 @@ Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=9], partitionBy=[b], order +- Exchange(distribution=[forward]) +- Sort(orderBy=[b ASC, a ASC]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, rk)]]], fields=[a, b, rk]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, rk]) ]]> @@ -90,7 +90,7 @@ WHERE rk1 < 10 LogicalProject(a=[$0], b=[$1], rk1=[$2], rk2=[$3]) +- LogicalFilter(condition=[<($2, 10)]) +- LogicalProject(a=[$0], b=[$1], rk1=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)], rk2=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -104,7 +104,7 @@ Calc(select=[a, b, w0$o0 AS rk1, w0$o0 AS rk2]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[b ASC, a ASC]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -121,7 +121,7 @@ WHERE rk < 10 LogicalProject(a=[$0], rk=[$1], b=[$2], c=[$3]) +- LogicalFilter(condition=[<($1, 10)]) +- LogicalProject(a=[$0], rk=[RANK() OVER (PARTITION BY $0 ORDER BY $0 NULLS FIRST)], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -134,7 +134,7 @@ Calc(select=[a, w0$o0 AS rk, b, c]) +- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=9], partitionBy=[a], orderBy=[a ASC], global=[false], select=[a, b, c]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -151,7 +151,7 @@ WHERE rk = 2 LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[=($2, 2)]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST, $2 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -164,7 +164,7 @@ Calc(select=[a, b, 2 AS $2]) +- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=2], partitionBy=[b], orderBy=[a ASC, c ASC], global=[false], select=[a, b, c]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[b ASC, a ASC, c ASC]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -181,7 +181,7 @@ WHERE rk <= 2 AND rk > -2 LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[AND(<=($2, 2), >($2, -2))]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1, $2 ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -194,7 +194,7 @@ Calc(select=[a, b, w0$o0]) +- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=2], partitionBy=[b, c], orderBy=[a ASC], global=[false], select=[a, b, c]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[b ASC, c ASC, a ASC]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -211,7 +211,7 @@ WHERE rk <= 2 AND a > 10 LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[AND(<=($2, 2), >($0, 10))]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -225,7 +225,7 @@ Calc(select=[a, b, w0$o0], where=[(a > 10)]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[b ASC, a ASC]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -242,7 +242,7 @@ WHERE rk < 10 LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[<($2, 10)]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -254,7 +254,7 @@ Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=9], partitionBy=[], orderB +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -279,7 +279,7 @@ LogicalProject(rn1=[CAST($3):INTEGER NOT NULL], rn2=[CAST($4):INTEGER NOT NULL]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rna=[$3], rnb=[ROW_NUMBER() OVER (PARTITION BY $0 ORDER BY $1 DESC NULLS LAST)]) +- LogicalFilter(condition=[<=($3, 100)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rna=[ROW_NUMBER() OVER (PARTITION BY $0, $2 ORDER BY $1 DESC NULLS LAST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -294,7 +294,7 @@ Calc(select=[CAST(rna AS INTEGER) AS rn1, CAST(w0$o0 AS INTEGER) AS rn2], where= +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC, c ASC, b DESC]) +- Exchange(distribution=[hash[a, c]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml index 7c13155b7675c..fa0b82c9cf43e 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml @@ -24,9 +24,9 @@ limitations under the License. @@ -36,10 +36,10 @@ HashAggregate(isMerge=[false], groupBy=[c], select=[c]) +- HashJoin(joinType=[LeftSemiJoin], where=[IS NOT DISTINCT FROM(c, f)], select=[c], build=[right]) :- Exchange(distribution=[hash[c]]) : +- Calc(select=[c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a, b, c]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[f]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[d, e, f]) ]]> @@ -51,9 +51,9 @@ HashAggregate(isMerge=[false], groupBy=[c], select=[c]) @@ -66,9 +66,9 @@ Calc(select=[c0 AS c]) +- LocalHashAggregate(groupBy=[c], select=[c, Partial_COUNT(vcol_left_marker) AS count$0, Partial_COUNT(vcol_right_marker) AS count$1]) +- Union(all=[true], union=[c, vcol_left_marker, vcol_right_marker]) :- Calc(select=[c, true AS vcol_left_marker, null:BOOLEAN AS vcol_right_marker]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a, b, c]) +- Calc(select=[f, null:BOOLEAN AS vcol_left_marker, true AS vcol_right_marker]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[d, e, f]) ]]> @@ -81,9 +81,9 @@ Calc(select=[c0 AS c]) LogicalIntersect(all=[false]) :- LogicalProject(c=[$2]) : +- LogicalFilter(condition=[=(1, 0)]) -: +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -100,10 +100,10 @@ Values(tuples=[[]], values=[c]) @@ -121,9 +121,9 @@ Values(tuples=[[]], values=[c]) LogicalProject(a=[$0]) +- LogicalIntersect(all=[false]) :- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -134,10 +134,10 @@ Calc(select=[a]) +- HashJoin(joinType=[LeftSemiJoin], where=[(((a = d) OR (a IS NULL AND d IS NULL)) AND ((b = e) OR (b IS NULL AND e IS NULL)))], select=[a, b], build=[right]) :- Exchange(distribution=[hash[a, b]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, e]]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[d, e, f]) ]]> @@ -149,9 +149,9 @@ Calc(select=[a]) @@ -161,10 +161,10 @@ HashAggregate(isMerge=[false], groupBy=[c], select=[c]) +- HashJoin(joinType=[LeftAntiJoin], where=[IS NOT DISTINCT FROM(c, f)], select=[c], build=[right]) :- Exchange(distribution=[hash[c]]) : +- Calc(select=[c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a, b, c]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[f]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[d, e, f]) ]]> @@ -176,9 +176,9 @@ HashAggregate(isMerge=[false], groupBy=[c], select=[c]) @@ -191,9 +191,9 @@ Calc(select=[c0 AS c]) +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(vcol_marker) AS sum$0]) +- Union(all=[true], union=[c, vcol_marker]) :- Calc(select=[c, 1 AS vcol_marker]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a, b, c]) +- Calc(select=[f, -1 AS vcol_marker]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[d, e, f]) ]]> @@ -206,9 +206,9 @@ Calc(select=[c0 AS c]) LogicalMinus(all=[false]) :- LogicalProject(c=[$2]) : +- LogicalFilter(condition=[=(1, 0)]) -: +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -225,10 +225,10 @@ Values(tuples=[[]], values=[c]) @@ -237,7 +237,7 @@ HashAggregate(isMerge=[true], groupBy=[c], select=[c]) +- Exchange(distribution=[hash[c]]) +- LocalHashAggregate(groupBy=[c], select=[c]) +- Calc(select=[c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a, b, c]) ]]> @@ -249,9 +249,9 @@ HashAggregate(isMerge=[true], groupBy=[c], select=[c]) @@ -260,7 +260,7 @@ HashAggregate(isMerge=[false], groupBy=[a, b, c], select=[a, b, c]) +- Exchange(distribution=[keep_input_as_is[hash[a, b, c]]]) +- HashJoin(joinType=[LeftAntiJoin], where=[(((a = a0) OR (a IS NULL AND a0 IS NULL)) AND ((b = b0) OR (b IS NULL AND b0 IS NULL)) AND ((c = c0) OR (c IS NULL AND c0 IS NULL)))], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[a, b, c]], shuffle_mode=[BATCH]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])(reuse_id=[1]) +- Exchange(distribution=[hash[a, b, c]]) +- Reused(reference_id=[1]) ]]> @@ -274,16 +274,16 @@ HashAggregate(isMerge=[false], groupBy=[a, b, c], select=[a, b, c]) ($2, 0), $1, null:RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) CHARACTER SET "UTF-16LE" _2))]) - +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, A]]) ]]> 0), b, null:RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2)) AS EXPR$0]) +- Reused(reference_id=[1]) ]]> @@ -297,16 +297,16 @@ Union(all=[true], union=[a]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.xml index aac313e1f6299..6fdbcc3a8e2e0 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.xml @@ -24,7 +24,7 @@ limitations under the License. @@ -32,7 +32,7 @@ LogicalSort(sort0=[$0], sort1=[$1], dir0=[DESC-nulls-last], dir1=[ASC-nulls-firs SortLimit(orderBy=[a DESC, b ASC], offset=[1], fetch=[10], global=[true]) +- Exchange(distribution=[single]) +- SortLimit(orderBy=[a DESC, b ASC], offset=[0], fetch=[11], global=[false]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -44,7 +44,7 @@ SortLimit(orderBy=[a DESC, b ASC], offset=[1], fetch=[10], global=[true]) @@ -61,14 +61,14 @@ Values(tuples=[[]], values=[a, b, c]) @@ -80,7 +80,7 @@ SortLimit(orderBy=[a DESC], offset=[10], fetch=[unlimited], global=[true]) @@ -97,7 +97,7 @@ Values(tuples=[[]], values=[a, b, c]) @@ -105,7 +105,7 @@ LogicalSort(sort0=[$0], dir0=[DESC-nulls-last], fetch=[5]) SortLimit(orderBy=[a DESC], offset=[0], fetch=[5], global=[true]) +- Exchange(distribution=[single]) +- SortLimit(orderBy=[a DESC], offset=[0], fetch=[5], global=[false]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -117,14 +117,14 @@ SortLimit(orderBy=[a DESC], offset=[0], fetch=[5], global=[true]) @@ -136,7 +136,7 @@ SortLimit(orderBy=[a DESC], offset=[10], fetch=[unlimited], global=[true]) @@ -153,7 +153,7 @@ Values(tuples=[[]], values=[a, b, c]) @@ -161,7 +161,7 @@ LogicalSort(sort0=[$0], dir0=[DESC-nulls-last], fetch=[5]) SortLimit(orderBy=[a DESC], offset=[0], fetch=[5], global=[true]) +- Exchange(distribution=[single]) +- SortLimit(orderBy=[a DESC], offset=[0], fetch=[5], global=[false]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -173,7 +173,7 @@ SortLimit(orderBy=[a DESC], offset=[0], fetch=[5], global=[true]) @@ -181,7 +181,7 @@ LogicalSort(sort0=[$0], sort1=[$1], dir0=[DESC-nulls-last], dir1=[ASC-nulls-firs SortLimit(orderBy=[a DESC, b ASC], offset=[1], fetch=[10], global=[true]) +- Exchange(distribution=[single]) +- SortLimit(orderBy=[a DESC, b ASC], offset=[0], fetch=[11], global=[false]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -193,7 +193,7 @@ SortLimit(orderBy=[a DESC, b ASC], offset=[1], fetch=[10], global=[true]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortTest.xml index 1b0be44aa16da..1a8d48861b942 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortTest.xml @@ -24,14 +24,14 @@ limitations under the License. @@ -43,7 +43,7 @@ Sort(orderBy=[a DESC, b ASC]) @@ -51,7 +51,7 @@ LogicalSort(sort0=[$0], dir0=[DESC-nulls-last]) SortLimit(orderBy=[a DESC], offset=[0], fetch=[200], global=[true]) +- Exchange(distribution=[single]) +- SortLimit(orderBy=[a DESC], offset=[0], fetch=[200], global=[false]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -63,14 +63,14 @@ SortLimit(orderBy=[a DESC], offset=[0], fetch=[200], global=[true]) @@ -82,14 +82,14 @@ Sort(orderBy=[a DESC]) @@ -101,7 +101,7 @@ Sort(orderBy=[a DESC]) @@ -109,7 +109,7 @@ LogicalSort(sort0=[$0], dir0=[DESC-nulls-last]) SortLimit(orderBy=[a DESC], offset=[0], fetch=[200], global=[true]) +- Exchange(distribution=[single]) +- SortLimit(orderBy=[a DESC], offset=[0], fetch=[200], global=[false]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.xml index 6cf86314772a7..281a28978e7c9 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.xml @@ -30,10 +30,10 @@ LogicalProject(a=[$0]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalSort(fetch=[10]) : +- LogicalProject(a=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalSort(fetch=[10]) +- LogicalProject(a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -45,7 +45,7 @@ Calc(select=[a]) : +- Limit(offset=[0], fetch=[10], global=[true]) : +- Exchange(distribution=[single]) : +- Limit(offset=[0], fetch=[10], global=[false]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Reused(reference_id=[1]) ]]> @@ -65,10 +65,10 @@ LogicalProject(a=[$0]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalSort(fetch=[10]) : +- LogicalProject(a=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalSort(fetch=[10]) +- LogicalProject(a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -80,7 +80,7 @@ Calc(select=[a]) : +- Limit(offset=[0], fetch=[10], global=[true]) : +- Exchange(distribution=[single]) : +- Limit(offset=[0], fetch=[10], global=[false]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[any], shuffle_mode=[BATCH]) +- Reused(reference_id=[1]) ]]> @@ -101,13 +101,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], a0=[$6], b0=[$7], :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) : +- LogicalFilter(condition=[=($0, $3)]) : +- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[=($0, $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -117,18 +117,18 @@ HashJoin(joinType=[InnerJoin], where=[(b = e0)], select=[a, b, c, d, e, f, a0, b : +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], build=[left]) : :- Exchange(distribution=[hash[a]]) : : +- Calc(select=[a, b, c], where=[(a < 10)]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, e, f], where=[(d < 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) +- Exchange(distribution=[hash[e]]) +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], build=[left]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[(a > 5)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, e, f], where=[(d > 5)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) ]]> @@ -196,13 +196,13 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3], a0=[$4], b0=[$5], d0=[$6], e0=[$7 :- LogicalProject(a=[$0], b=[$1], d=[$3], e=[$4]) : +- LogicalFilter(condition=[=($0, $3)]) : +- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[$1], d=[$3], e=[$4]) +- LogicalFilter(condition=[=($0, $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -212,10 +212,10 @@ HashJoin(joinType=[InnerJoin], where=[(b = e0)], select=[a, b, d, e, a0, b0, d0, : +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[left]) : :- Exchange(distribution=[hash[a]]) : : +- Calc(select=[a, b], where=[(a < 10)]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])(reuse_id=[1]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c])(reuse_id=[1]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, e], where=[(d < 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])(reuse_id=[2]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f])(reuse_id=[2]) +- Exchange(distribution=[hash[e]]) +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[left]) :- Exchange(distribution=[hash[a]]) @@ -320,22 +320,22 @@ LogicalProject(c=[$0], e=[$1], avg_b=[$2], sum_b=[$3], psum=[$4], nsum=[$5], avg : : +- LogicalProject(c=[$2], e=[$4], b=[$1]) : : +- LogicalFilter(condition=[AND(=($0, $3), IS NOT NULL($2), >($4, 10))]) : : +- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) : +- LogicalProject(sum_b=[$2], avg_b=[/(CASE(>(COUNT($2) OVER (PARTITION BY $0, $1), 0), $SUM0($2) OVER (PARTITION BY $0, $1), null:BIGINT), COUNT($2) OVER (PARTITION BY $0, $1))], rn=[RANK() OVER (PARTITION BY $0, $1 ORDER BY $0 NULLS FIRST, $1 NULLS FIRST)], c=[$0], e=[$1]) : +- LogicalAggregate(group=[{0, 1}], sum_b=[SUM($2)]) : +- LogicalProject(c=[$2], e=[$4], b=[$1]) : +- LogicalFilter(condition=[AND(=($0, $3), IS NOT NULL($2), >($4, 10))]) : +- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(sum_b=[$2], avg_b=[/(CASE(>(COUNT($2) OVER (PARTITION BY $0, $1), 0), $SUM0($2) OVER (PARTITION BY $0, $1), null:BIGINT), COUNT($2) OVER (PARTITION BY $0, $1))], rn=[RANK() OVER (PARTITION BY $0, $1 ORDER BY $0 NULLS FIRST, $1 NULLS FIRST)], c=[$0], e=[$1]) +- LogicalAggregate(group=[{0, 1}], sum_b=[SUM($2)]) +- LogicalProject(c=[$2], e=[$4], b=[$1]) +- LogicalFilter(condition=[AND(=($0, $3), IS NOT NULL($2), >($4, 10))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -355,10 +355,10 @@ Calc(select=[c, e, avg_b, sum_b, sum_b0 AS psum, sum_b1 AS nsum, avg_b0 AS avg_b : +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e], build=[right]) : :- Exchange(distribution=[hash[a]]) : : +- Calc(select=[a, b, c], where=[c IS NOT NULL]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, e], where=[(e > 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) :- Exchange(distribution=[hash[c, e, rn]]) : +- Calc(select=[sum_b, (CASE((w0$o0 > 0), w0$o1, null:BIGINT) / w0$o0) AS avg_b, w1$o0 AS rn, c, e], where=[((c <> '') AND ((sum_b - (CASE((w0$o0 > 0), w0$o1, null:BIGINT) / w0$o0)) > 3))]) : +- Reused(reference_id=[1]) @@ -516,10 +516,10 @@ LogicalProject(c=[$0], a=[$1], b=[$2], c0=[$3], a0=[$4], b0=[$5]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalAggregate(group=[{0}], a=[MyFirst($1)], b=[MyLast($2)]) : +- LogicalProject(c=[$2], a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalAggregate(group=[{0}], a=[MyFirst($1)], b=[MyLast($2)]) +- LogicalProject(c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -532,7 +532,7 @@ Calc(select=[c, a, b, c0, a1, b0]) : +- Exchange(distribution=[forward]) : +- Sort(orderBy=[c ASC]) : +- Exchange(distribution=[hash[c]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[c, a, b], where=[(a > 1)]) +- Reused(reference_id=[1]) @@ -556,15 +556,15 @@ LogicalUnion(all=[true]) : +- LogicalJoin(condition=[true], joinType=[inner]) : :- LogicalProject(a=[$0], b=[$1], c=[$2]) : : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'test%')]) -: : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) -: +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) +: : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], c=[LOWER($2)], e=[$4]) +- LogicalFilter(condition=[=($0, $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'test%')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -572,10 +572,10 @@ LogicalUnion(all=[true]) MultipleInput(readOrder=[0,1], members=[\nUnion(all=[true], union=[a, c, e])\n:- Calc(select=[a, LOWER(c) AS c, e])(reuse_id=[1])\n: +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, c, d, e], build=[left])\n: :- [#1] Exchange(distribution=[hash[a]])\n: +- [#2] Exchange(distribution=[hash[d]])\n+- Reused(reference_id=[1])\n]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, c], where=[LIKE(c, 'test%')]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) ]]> @@ -592,17 +592,17 @@ UNION ALL LogicalUnion(all=[true]) :- LogicalProject(a=[$0], EXPR$1=[random_udf()]) : +- LogicalFilter(condition=[>($0, 10)]) -: +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0], EXPR$1=[random_udf()]) +- LogicalFilter(condition=[>($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> 10)])(reuse_id=[1]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Reused(reference_id=[1]) ]]> @@ -620,17 +620,17 @@ UNION ALL LogicalUnion(all=[true]) :- LogicalProject(a=[$0]) : +- LogicalFilter(condition=[>($1, random_udf($0))]) -: +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0]) +- LogicalFilter(condition=[>($1, random_udf($0))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> random_udf(a))])(reuse_id=[1]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Reused(reference_id=[1]) ]]> @@ -649,11 +649,11 @@ LogicalProject(a=[$0], b=[$1], c=[$2], v=[$3], a0=[$4], b0=[$5], c0=[$6], v0=[$7 +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], v=[$3]) : +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalTableFunctionScan(invocation=[str_split($cor0.c, _UTF-16LE'-')], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], v=[$3]) +- LogicalCorrelate(correlation=[$cor1], joinType=[inner], requiredColumns=[{2}]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalTableFunctionScan(invocation=[str_split($cor1.c, _UTF-16LE'-')], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) ]]> @@ -662,10 +662,10 @@ LogicalProject(a=[$0], b=[$1], c=[$2], v=[$3], a0=[$4], b0=[$5], c0=[$6], v0=[$7 HashJoin(joinType=[InnerJoin], where=[(EXPR$0 = EXPR$00)], select=[a, b, c, EXPR$0, a0, b0, c0, EXPR$00], build=[left]) :- Exchange(distribution=[hash[EXPR$0]]) : +- Correlate(invocation=[str_split($cor0.c, _UTF-16LE'-')], correlate=[table(str_split($cor0.c,'-'))], select=[a,b,c,EXPR$0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) EXPR$0)], joinType=[INNER]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[EXPR$0]]) +- Correlate(invocation=[str_split($cor1.c, _UTF-16LE'-')], correlate=[table(str_split($cor1.c,'-'))], select=[a,b,c,EXPR$0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) EXPR$0)], joinType=[INNER]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) ]]> @@ -683,11 +683,11 @@ LogicalProject(a=[$0], b=[$1], c=[$2], s=[$3], a0=[$4], b0=[$5], c0=[$6], s0=[$7 +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], s=[$3]) : +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalTableFunctionScan(invocation=[TableFun($cor0.c)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor1], joinType=[inner], requiredColumns=[{2}]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalTableFunctionScan(invocation=[TableFun($cor1.c)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) ]]> @@ -696,9 +696,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2], s=[$3], a0=[$4], b0=[$5], c0=[$6], s0=[$7 MultipleInput(readOrder=[1,0], members=[\nHashJoin(joinType=[InnerJoin], where=[(c = EXPR$00)], select=[a, b, c, EXPR$0, a0, b0, c0, EXPR$00], build=[left])\n:- Correlate(invocation=[TableFun($cor0.c)], correlate=[table(TableFun($cor0.c))], select=[a,b,c,EXPR$0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) EXPR$0)], joinType=[INNER])\n: +- [#2] Exchange(distribution=[hash[c]])\n+- [#1] Exchange(distribution=[hash[EXPR$0]])\n]) :- Exchange(distribution=[hash[EXPR$0]]) : +- Correlate(invocation=[TableFun($cor1.c)], correlate=[table(TableFun($cor1.c))], select=[a,b,c,EXPR$0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) EXPR$0)], joinType=[INNER]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[c]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) ]]> @@ -716,10 +716,10 @@ LogicalProject(a=[$0], b=[$1], a0=[$2]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalSort(fetch=[10]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalSort(fetch=[10]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -731,7 +731,7 @@ Calc(select=[a, b, a1 AS a0]) : +- Limit(offset=[0], fetch=[10], global=[true])(reuse_id=[1]) : +- Exchange(distribution=[single]) : +- Limit(offset=[0], fetch=[10], global=[false]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[a, b]) +- Reused(reference_id=[1]) @@ -784,15 +784,15 @@ LogicalUnion(all=[true]) : +- LogicalJoin(condition=[true], joinType=[inner]) : :- LogicalProject(a=[$0], b=[$1], c=[$2]) : : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'test%')]) -: : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) -: +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) +: : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), <>($5, _UTF-16LE''))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'test%')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -800,14 +800,14 @@ LogicalUnion(all=[true]) MultipleInput(readOrder=[0,1,0,1], members=[\nUnion(all=[true], union=[a, b, c, d, e, f])\n:- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], build=[left])\n: :- [#1] Exchange(distribution=[hash[a]])\n: +- [#2] Exchange(distribution=[hash[d]])\n+- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], build=[left])\n :- [#1] Exchange(distribution=[hash[a]])\n +- [#4] Exchange(distribution=[hash[d]])\n]) :- Exchange(distribution=[hash[a]])(reuse_id=[1]) : +- Calc(select=[a, b, c], where=[LIKE(c, 'test%')]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) :- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, e, f], where=[(e > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) :- Reused(reference_id=[1]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, e, f], where=[(f <> '')]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) ]]> @@ -825,10 +825,10 @@ LogicalProject(c=[$0], a=[$1], b=[$2], c0=[$3], a0=[$4], b0=[$5]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalAggregate(group=[{0}], a=[SUM($1)], b=[SUM($2)]) : +- LogicalProject(c=[$2], a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalAggregate(group=[{0}], a=[SUM($1)], b=[SUM($2)]) +- LogicalProject(c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -840,7 +840,7 @@ Calc(select=[c, a, b, c0, a1, b0]) : +- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_SUM(sum$0) AS a, Final_SUM(sum$1) AS b])(reuse_id=[1]) : +- Exchange(distribution=[hash[c]]) : +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(a) AS sum$0, Partial_SUM(b) AS sum$1]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[c, a, b], where=[(a > 1)]) +- Reused(reference_id=[1]) @@ -862,13 +862,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], a0=[$6], b0=[$7], :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) : +- LogicalFilter(condition=[AND(=($0, $3), LIKE($2, _UTF-16LE'He%'))]) : +- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), LIKE($2, _UTF-16LE'He%'))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -878,9 +878,9 @@ HashJoin(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, e, f, a0, b : +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], build=[left])(reuse_id=[1]) : :- Exchange(distribution=[hash[a]]) : : +- Calc(select=[a, b, c], where=[LIKE(c, 'He%')]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) +- Exchange(distribution=[hash[d]], shuffle_mode=[BATCH]) +- Reused(reference_id=[1]) ]]> @@ -902,13 +902,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], a0=[$6], b0=[$7], :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) : +- LogicalFilter(condition=[AND(>($1, 1), <($4, 2))]) : +- LogicalJoin(condition=[OR(=(ABS($0), ABS($3)), =($2, $5))], joinType=[full]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(>($1, 1), <($4, 2))]) +- LogicalJoin(condition=[OR(=(ABS($0), ABS($3)), =($2, $5))], joinType=[full]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -920,9 +920,9 @@ Calc(select=[a, b, c, d, e, f, a1, b0, c0, d0, e0, f0]) : +- NestedLoopJoin(joinType=[InnerJoin], where=[((ABS(a) = ABS(d)) OR (c = f))], select=[a, b, c, d, e, f], build=[left])(reuse_id=[1]) : :- Exchange(distribution=[broadcast]) : : +- Calc(select=[a, b, c], where=[(b > 1)]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) : +- Calc(select=[d, e, f], where=[(e < 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) +- Exchange(distribution=[hash[b]]) +- Reused(reference_id=[1]) ]]> @@ -944,13 +944,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], a0=[$6], b0=[$7], :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) : +- LogicalFilter(condition=[AND(>($1, 1), <($4, 2))]) : +- LogicalJoin(condition=[OR(=(random_udf($0), random_udf($3)), =($2, $5))], joinType=[full]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(>($1, 1), <($4, 2))]) +- LogicalJoin(condition=[OR(=(random_udf($0), random_udf($3)), =($2, $5))], joinType=[full]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -962,9 +962,9 @@ Calc(select=[a, b, c, d, e, f, a1, b0, c0, d0, e0, f0]) : +- NestedLoopJoin(joinType=[InnerJoin], where=[((random_udf(a) = random_udf(d)) OR (c = f))], select=[a, b, c, d, e, f], build=[left])(reuse_id=[1]) : :- Exchange(distribution=[broadcast]) : : +- Calc(select=[a, b, c], where=[(b > 1)]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) : +- Calc(select=[d, e, f], where=[(e < 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) +- Exchange(distribution=[hash[b]]) +- Reused(reference_id=[1]) ]]> @@ -984,10 +984,10 @@ LogicalProject(a=[$0], b=[$1], a0=[$2]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalSort(fetch=[10]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalSort(fetch=[10]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -999,7 +999,7 @@ Calc(select=[a, b, a1 AS a0]) : +- Limit(offset=[0], fetch=[10], global=[true])(reuse_id=[1]) : +- Exchange(distribution=[single]) : +- Limit(offset=[0], fetch=[10], global=[false]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[a, b]) +- Reused(reference_id=[1]) @@ -1021,13 +1021,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], a0=[$6], b0=[$7], :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) : +- LogicalFilter(condition=[AND(=($0, $3), LIKE($2, _UTF-16LE'He%'))]) : +- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), LIKE($2, _UTF-16LE'He%'))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1037,8 +1037,8 @@ NestedLoopJoin(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, e, f, : +- NestedLoopJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], build=[left])(reuse_id=[1]) : :- Exchange(distribution=[broadcast]) : : +- Calc(select=[a, b, c], where=[LIKE(c, 'He%')]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) +- Exchange(distribution=[any], shuffle_mode=[BATCH]) +- Reused(reference_id=[1]) ]]> @@ -1057,9 +1057,9 @@ LogicalProject(a=[$0], b=[$1], EXPR$2=[$2], a0=[$3], b0=[$4], EXPR$20=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), <($1, 100), >($4, 10))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], EXPR$2=[MyFirst($2) OVER (PARTITION BY $2 ORDER BY $2 DESC NULLS LAST)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0], b=[$1], EXPR$2=[MyFirst($2) OVER (PARTITION BY $2 ORDER BY $2 DESC NULLS LAST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1071,7 +1071,7 @@ HashJoin(joinType=[InnerJoin], where=[(a = a0)], select=[a, b, w0$o0, a0, b0, w0 : +- Exchange(distribution=[forward]) : +- Sort(orderBy=[c DESC]) : +- Exchange(distribution=[hash[c]]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, w0$o0], where=[(b > 10)]) +- Reused(reference_id=[1]) @@ -1092,10 +1092,10 @@ LogicalProject(c=[$0], a=[$1], b=[$2], c0=[$3], a0=[$4], b0=[$5]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalAggregate(group=[{0}], a=[SUM($1)], b=[SUM($2)]) : +- LogicalProject(c=[$2], a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalAggregate(group=[{0}], a=[SUM($1)], b=[SUM($2)]) +- LogicalProject(c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1106,7 +1106,7 @@ HashJoin(joinType=[InnerJoin], where=[(a = a0)], select=[c, a, b, c0, a0, b0], b : +- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_SUM(sum$0) AS a, Final_SUM(sum$1) AS b])(reuse_id=[1]) : +- Exchange(distribution=[hash[c]]) : +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(a) AS sum$0, Partial_SUM(b) AS sum$1]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[c, a, b], where=[((b < 10) AND (a > 1))]) +- Reused(reference_id=[1]) @@ -1127,10 +1127,10 @@ LogicalProject(c=[$0], a=[$1], b=[$2], c0=[$3], a0=[$4], b0=[$5]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalAggregate(group=[{0}], a=[SUM($1)], b=[SUM($2)]) : +- LogicalProject(c=[$2], a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalAggregate(group=[{0}], a=[SUM($1)], b=[SUM($2)]) +- LogicalProject(c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1146,7 +1146,7 @@ Calc(select=[c, a, b, c0, a1, b0]) : +- LocalSortAggregate(groupBy=[c], select=[c, Partial_SUM(a) AS sum$0, Partial_SUM(b) AS sum$1]) : +- Exchange(distribution=[forward]) : +- Sort(orderBy=[c ASC]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[c, a, b], where=[(a > 1)]) +- Reused(reference_id=[1]) @@ -1166,9 +1166,9 @@ LogicalProject(a=[$0], b=[$1], EXPR$2=[$2], a0=[$3], b0=[$4], EXPR$20=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), <($1, 100), >($4, 10))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], EXPR$2=[RANK() OVER (ORDER BY $2 DESC NULLS LAST)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0], b=[$1], EXPR$2=[RANK() OVER (ORDER BY $2 DESC NULLS LAST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1179,7 +1179,7 @@ HashJoin(joinType=[InnerJoin], where=[(a = a0)], select=[a, b, w0$o0, a0, b0, w0 : +- OverAggregate(orderBy=[c DESC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0])(reuse_id=[1]) : +- Sort(orderBy=[c DESC]) : +- Exchange(distribution=[single]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, w0$o0], where=[(b > 10)]) +- Reused(reference_id=[1]) @@ -1238,11 +1238,11 @@ LogicalProject(c=[$0], a=[$1], b=[$2], c0=[$3], a0=[$4], b0=[$5]) :- LogicalSort(sort0=[$1], sort1=[$2], dir0=[ASC-nulls-first], dir1=[DESC-nulls-last], fetch=[10]) : +- LogicalAggregate(group=[{0}], a=[SUM($1)], b=[SUM($2)]) : +- LogicalProject(c=[$2], a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalSort(sort0=[$1], sort1=[$2], dir0=[ASC-nulls-first], dir1=[DESC-nulls-last], fetch=[10]) +- LogicalAggregate(group=[{0}], a=[SUM($1)], b=[SUM($2)]) +- LogicalProject(c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1256,7 +1256,7 @@ NestedLoopJoin(joinType=[InnerJoin], where=[(a = a0)], select=[c, a, b, c0, a0, : +- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_SUM(sum$0) AS a, Final_SUM(sum$1) AS b]) : +- Exchange(distribution=[hash[c]]) : +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(a) AS sum$0, Partial_SUM(b) AS sum$1]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[c, a, b], where=[((b < 10) AND (a > 1))]) +- Reused(reference_id=[1]) @@ -1278,17 +1278,17 @@ LogicalProject(a=[$0], c=[$1], c0=[$3]) :- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], c=[$2]) : : +- LogicalFilter(condition=[>($1, 10)]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(d=[$0], f=[$2]) : +- LogicalFilter(condition=[<($1, 100)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], c=[$2]) : +- LogicalFilter(condition=[>($1, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(d=[$0], f=[$2]) +- LogicalFilter(condition=[<($1, 100)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1298,9 +1298,9 @@ Calc(select=[a, c, c0]) :- Exchange(distribution=[hash[a]]) : +- Union(all=[true], union=[a, c])(reuse_id=[1]) : :- Calc(select=[a, c], where=[(b > 10)]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) : +- Calc(select=[d, f], where=[(e < 100)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) +- Exchange(distribution=[hash[a]], shuffle_mode=[BATCH]) +- Reused(reference_id=[1]) ]]> @@ -1321,13 +1321,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], a0=[$6], b0=[$7], :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) : +- LogicalFilter(condition=[AND(=($0, $3), LIKE($2, _UTF-16LE'He%'))]) : +- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), LIKE($2, _UTF-16LE'He%'))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1335,9 +1335,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], a0=[$6], b0=[$7], MultipleInput(members=[\nSortMergeJoin(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, e, f, a0, b0, c0, d0, e0, f0])\n:- SortMergeJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f])(reuse_id=[1])\n: :- [#1] Exchange(distribution=[hash[a]])\n: +- [#2] Exchange(distribution=[hash[d]])\n+- Reused(reference_id=[1])\n]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[LIKE(c, 'He%')]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) ]]> @@ -1349,15 +1349,15 @@ LogicalIntersect(all=[false]) : :- LogicalProject(random=[$0]) : : +- LogicalSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[1]) : : +- LogicalProject(random=[$0], EXPR$1=[RAND()]) -: : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) +: : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(random=[$0]) : +- LogicalSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[1]) : +- LogicalProject(random=[$0], EXPR$1=[RAND()]) -: +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(random=[$0]) +- LogicalSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[1]) +- LogicalProject(random=[$0], EXPR$1=[RAND()]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1369,7 +1369,7 @@ MultipleInput(readOrder=[0,1], members=[\nNestedLoopJoin(joinType=[LeftSemiJoin] : +- Exchange(distribution=[single]) : +- SortLimit(orderBy=[EXPR$1 ASC], offset=[0], fetch=[1], global=[false]) : +- Calc(select=[a AS random, RAND() AS EXPR$1]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[random]]) +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[IS NOT DISTINCT FROM(random, random0)], select=[random], build=[right]) :- Exchange(distribution=[any], shuffle_mode=[BATCH]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.xml index 9446dc78fe95c..c9c7cc051e9d6 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.xml @@ -85,22 +85,6 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) - - - - - - - - - - - diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnionTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnionTest.xml index 8516330534422..7aa823d1c2361 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnionTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnionTest.xml @@ -35,11 +35,11 @@ LogicalProject(a=[$0], c=[$1]) +- LogicalUnion(all=[true]) :- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], c=[$2]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) : +- LogicalProject(a=[$0], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) +- LogicalProject(a=[$0], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3]]) ]]> @@ -47,11 +47,11 @@ LogicalProject(a=[$0], c=[$1]) Union(all=[true], union=[a, c]) :- Union(all=[true], union=[a, c]) : :- Calc(select=[a, c], where=[(a > 2)]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) : +- Calc(select=[a, c], where=[(a > 2)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c]) +- Calc(select=[a, c], where=[(a > 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable3]], fields=[a, b, c]) ]]> @@ -69,18 +69,18 @@ SELECT * FROM ( 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, source: [TestTableSource(a, b, c)]]]), rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)] + : +- 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)] - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]), rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)] + +- 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/batch/sql/UnnestTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml index a5b851e8c34fb..46cdbd1e6dd43 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml @@ -43,7 +43,7 @@ Correlate(invocation=[$UNNEST_ROWS$1(ARRAY(1, 2, 3))], correlate=[table($UNNEST_ @@ -107,7 +107,7 @@ Calc(select=[b, f0 AS s]) @@ -138,7 +138,7 @@ LogicalProject(a=[$0], b=[$1], x=[$2], y=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalProject(a=[$0], b=[$1]) : +- LogicalFilter(condition=[<($0, 3)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(x=[$0], y=[$1]) +- Uncollect +- LogicalProject(b=[$cor0.b]) @@ -150,7 +150,7 @@ LogicalProject(a=[$0], b=[$1], x=[$2], y=[$3]) Calc(select=[a, b, _1, _2], where=[(_1 > a)]) +- Correlate(invocation=[$UNNEST_ROWS$1($cor0.b)], correlate=[table($UNNEST_ROWS$1($cor0.b))], select=[a,b,_1,_2], rowType=[RecordType(INTEGER a, RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2) ARRAY b, INTEGER _1, VARCHAR(2147483647) _2)], joinType=[INNER]) +- Calc(select=[a, b], where=[(a < 3)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]], fields=[a, b]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> @@ -199,7 +199,7 @@ Calc(select=[a, f0 AS s]) @@ -259,7 +259,7 @@ Calc(select=[b, _1 AS id, _2 AS point]) @@ -283,7 +283,7 @@ Calc(select=[a, b, f0 AS s]) LogicalProject(a=[$0], b=[$1], s=[$2], t=[$3]) +- LogicalFilter(condition=[>($2, 13)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(s=[$0], t=[$1]) +- Uncollect +- LogicalProject(b=[$cor0.b]) @@ -293,7 +293,7 @@ LogicalProject(a=[$0], b=[$1], s=[$2], t=[$3]) ($0, 13)]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]], fields=[a, b]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> @@ -306,7 +306,7 @@ Correlate(invocation=[$UNNEST_ROWS$1($cor0.b)], correlate=[table($UNNEST_ROWS$1( LogicalProject(a=[$0], b=[$1], _1=[$2], _2=[$3]) +- LogicalFilter(condition=[>($2, 1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- Uncollect +- LogicalProject(b=[$cor0.b]) +- LogicalValues(tuples=[[{ 0 }]]) @@ -315,7 +315,7 @@ LogicalProject(a=[$0], b=[$1], _1=[$2], _2=[$3]) ($0, 1)]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]], fields=[a, b]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> @@ -340,7 +340,7 @@ LogicalProject(a=[$0], b1=[$1], b2=[$2]) +- LogicalFilter(condition=[>=($2, 12)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(b1=[$0], b2=[$1]) +- Uncollect +- LogicalProject(b=[$cor0.b]) @@ -351,7 +351,7 @@ LogicalProject(a=[$0], b1=[$1], b2=[$2]) =($0, 12), <>($1, _UTF-16LE'Hello'))]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]], fields=[a, b]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/WindowTableFunctionTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/WindowTableFunctionTest.xml index 7ef94b1552323..26ebd4d52f691 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/WindowTableFunctionTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/WindowTableFunctionTest.xml @@ -37,7 +37,7 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[SUM($3)]) +- LogicalProject(window_start=[$5], window_end=[$6], a=[$0], b=[$1]) +- LogicalTableFunctionScan(invocation=[TUMBLE(DESCRIPTOR($4), 3000:INTERVAL SECOND)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, INTEGER d, TIMESTAMP(3) ts, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], ts=[$4]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -52,7 +52,7 @@ HashAggregate(isMerge=[true], groupBy=[window_start, window_end, b], select=[win +- Calc(select=[window_start, window_end, a, b]) +- WindowTableFunction(window=[TUMBLE(time_col=[ts], size=[3 s])]) +- Calc(select=[a, b, ts]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c, d, ts]) ]]> @@ -69,13 +69,13 @@ FROM TABLE( LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], ts=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) +- LogicalTableFunctionScan(invocation=[CUMULATE(DESCRIPTOR($4), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, INTEGER d, TIMESTAMP(3) ts, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], ts=[$4]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -91,13 +91,13 @@ FROM TABLE(HOP(TABLE MyTable1, DESCRIPTOR(ts), INTERVAL '1' HOUR, INTERVAL '2' H LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], ts=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) +- LogicalTableFunctionScan(invocation=[HOP(DESCRIPTOR($4), 3600000:INTERVAL HOUR, 7200000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, INTEGER d, TIMESTAMP(3) ts, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], ts=[$4]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -113,13 +113,13 @@ FROM TABLE(TUMBLE(TABLE MyTable1, DESCRIPTOR(ts), INTERVAL '15' MINUTE)) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], ts=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) +- LogicalTableFunctionScan(invocation=[TUMBLE(DESCRIPTOR($4), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, INTEGER d, TIMESTAMP(3) ts, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], ts=[$4]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -139,7 +139,7 @@ LogicalProject(EXPR$0=[$3]) +- LogicalProject(window_start=[$5], window_end=[$6], a=[$0], c=[$2]) +- LogicalTableFunctionScan(invocation=[TUMBLE(DESCRIPTOR($4), 3000:INTERVAL SECOND)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, INTEGER d, TIMESTAMP(3) ts, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], ts=[$4]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -155,7 +155,7 @@ Calc(select=[EXPR$0]) +- Calc(select=[window_start, window_end, a, c]) +- WindowTableFunction(window=[TUMBLE(time_col=[ts], size=[3 s])]) +- Calc(select=[a, c, ts]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c, d, ts]) ]]> @@ -177,7 +177,7 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[MAX($3)]) +- LogicalProject(window_start=[$5], window_end=[$6], a=[$0], c=[$2]) +- LogicalTableFunctionScan(invocation=[TUMBLE(DESCRIPTOR($4), 3000:INTERVAL SECOND)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, INTEGER d, TIMESTAMP(3) ts, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], ts=[$4]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -192,7 +192,7 @@ SortAggregate(isMerge=[true], groupBy=[window_start, window_end, a], select=[win +- Calc(select=[window_start, window_end, a, c]) +- WindowTableFunction(window=[TUMBLE(time_col=[ts], size=[3 s])]) +- Calc(select=[a, c, ts]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c, d, ts]) ]]> @@ -218,11 +218,11 @@ LogicalProject(ts=[$0], a=[$1], b=[$2], c=[$3], window_start=[$4], window_end=[$ :- LogicalProject(ts=[$0], a=[$1], b=[$2], c=[$3], window_start=[$4], window_end=[$5], window_time=[$6]) : +- LogicalTableFunctionScan(invocation=[TUMBLE(DESCRIPTOR($0), 900000:INTERVAL MINUTE)], rowType=[RecordType(TIMESTAMP(3) ts, BIGINT a, INTEGER b, VARCHAR(2147483647) c, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) window_time)]) : +- LogicalProject(ts=[$0], a=[$1], b=[$2], c=[$3]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(ts, a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], ts=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) +- LogicalTableFunctionScan(invocation=[TUMBLE(DESCRIPTOR($4), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, INTEGER d, TIMESTAMP(3) ts, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], ts=[$4]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -230,10 +230,10 @@ LogicalProject(ts=[$0], a=[$1], b=[$2], c=[$3], window_start=[$4], window_end=[$ HashJoin(joinType=[InnerJoin], where=[((window_start = window_start0) AND (window_end = window_end0) AND (a = b0))], select=[ts, a, b, c, window_start, window_end, window_time, a0, b0, c0, d, ts0, window_start0, window_end0, window_time0], build=[left]) :- Exchange(distribution=[hash[window_start, window_end, a]]) : +- WindowTableFunction(window=[TUMBLE(time_col=[ts], size=[15 min])]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[ts, a, b, c]) +- Exchange(distribution=[hash[window_start, window_end, b]]) +- WindowTableFunction(window=[TUMBLE(time_col=[ts], size=[15 min])]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c, d, ts]) ]]> @@ -256,7 +256,7 @@ LogicalProject(ts=[$0], a=[$1], b=[$2], c=[$3], window_start=[$4], window_end=[$ +- LogicalProject(ts=[$0], a=[$1], b=[$2], c=[$3], window_start=[$4], window_end=[$5], window_time=[$6], rownum=[RANK() OVER (PARTITION BY $1, $4, $5 ORDER BY $2 NULLS FIRST)]) +- LogicalTableFunctionScan(invocation=[TUMBLE(DESCRIPTOR($0), 900000:INTERVAL MINUTE)], rowType=[RecordType(TIMESTAMP(3) ts, BIGINT a, INTEGER b, VARCHAR(2147483647) c, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) window_time)]) +- LogicalProject(ts=[$0], a=[$1], b=[$2], c=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -269,7 +269,7 @@ Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=3], partitionBy=[a, window +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC, window_start ASC, window_end ASC, b ASC]) +- WindowTableFunction(window=[TUMBLE(time_col=[ts], size=[15 min])]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[ts, a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/DistinctAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/DistinctAggregateTest.xml index 4d372cab8f356..68bac9b28ec50 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/DistinctAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/DistinctAggregateTest.xml @@ -24,7 +24,7 @@ limitations under the License. @@ -39,7 +39,7 @@ Calc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2, EXPR$3]) +- LocalHashAggregate(groupBy=[a, b, $e], select=[a, b, $e, Partial_COUNT(a) AS count$0, Partial_SUM(b_0) AS sum$1]) +- Expand(projects=[{a, b, 0 AS $e, b AS b_0}, {a, null AS b, 1 AS $e, b AS b_0}]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -52,7 +52,7 @@ COUNT(DISTINCT d) FILTER (WHERE b > 1) FROM MyTable2 GROUP BY a]]> ($1, 1))], d=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -67,7 +67,7 @@ Calc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2]) +- LocalHashAggregate(groupBy=[a, $f2, d, $e], select=[a, $f2, d, $e, Partial_COUNT(c) FILTER $f2_0 AS count$0]) +- Expand(projects=[{a, c, $f2, d, 0 AS $e, $f2 AS $f2_0}, {a, c, null AS $f2, null AS d, 3 AS $e, $f2 AS $f2_0}]) +- Calc(select=[a, c, (b > 1) IS TRUE AS $f2, d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -80,7 +80,7 @@ Calc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2]) LogicalProject(EXPR$0=[$1], EXPR$1=[$2]) +- LogicalAggregate(group=[{0}], EXPR$0=[SUM(DISTINCT $1)], EXPR$1=[COUNT(DISTINCT $2)]) +- LogicalProject(b=[$1], a=[$0], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -94,7 +94,7 @@ Calc(select=[EXPR$0, EXPR$1]) +- Exchange(distribution=[hash[a, b, c, $e]]) +- LocalHashAggregate(groupBy=[a, b, c, $e], select=[a, b, c, $e]) +- Expand(projects=[{a, b, null AS c, 1 AS $e}, {null AS a, b, c, 4 AS $e}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -106,7 +106,7 @@ Calc(select=[EXPR$0, EXPR$1]) @@ -120,7 +120,7 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_SUM( +- LocalHashAggregate(groupBy=[a, b, $e], select=[a, b, $e, Partial_MAX(a_0) AS max$0, Partial_MIN(a_0) AS min$1]) +- Expand(projects=[{a, null AS b, 1 AS $e, a AS a_0}, {null AS a, b, 2 AS $e, a AS a_0}, {null AS a, null AS b, 3 AS $e, a AS a_0}]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -133,7 +133,7 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_SUM( LogicalProject(EXPR$0=[$1], EXPR$1=[$2], EXPR$2=[$3], EXPR$3=[$4]) +- LogicalAggregate(group=[{0}], EXPR$0=[COUNT(DISTINCT $1)], EXPR$1=[SUM(DISTINCT $2)], EXPR$2=[MAX($1)], EXPR$3=[MIN($1)]) +- LogicalProject(c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -147,7 +147,7 @@ Calc(select=[EXPR$0, EXPR$1, EXPR$2, EXPR$3]) +- Exchange(distribution=[hash[a, b, c, $e]]) +- LocalHashAggregate(groupBy=[a, b, c, $e], select=[a, b, c, $e, Partial_MAX(a_0) AS max$0, Partial_MIN(a_0) AS min$1]) +- Expand(projects=[{a, null AS b, c, 2 AS $e, a AS a_0}, {null AS a, b, c, 4 AS $e, a AS a_0}, {null AS a, null AS b, c, 6 AS $e, a AS a_0}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -160,7 +160,7 @@ COUNT(DISTINCT c) FILTER (WHERE a > 5) FROM MyTable]]> ($0, 5))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -174,7 +174,7 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_SUM( +- LocalHashAggregate(groupBy=[a, b, c, $f3, $e], select=[a, b, c, $f3, $e]) +- Expand(projects=[{a, null AS b, null AS c, null AS $f3, 7 AS $e}, {null AS a, b, null AS c, null AS $f3, 11 AS $e}, {null AS a, null AS b, c, $f3, 12 AS $e}]) +- Calc(select=[a, b, c, (a > 5) IS TRUE AS $f3]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -187,7 +187,7 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_SUM( LogicalProject(EXPR$0=[$2], EXPR$1=[$3]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}]], EXPR$0=[COUNT(DISTINCT $2)], EXPR$1=[SUM(DISTINCT $3)]) +- LogicalProject(c=[$2], d=[$3], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -203,7 +203,7 @@ Calc(select=[EXPR$0, EXPR$1]) +- Expand(projects=[{c, d, a, null AS b, $e, 2 AS $e_0}, {c, d, null AS a, b, $e, 4 AS $e_0}]) +- Expand(projects=[{c, null AS d, a, b, 1 AS $e}, {null AS c, d, a, b, 2 AS $e}]) +- Calc(select=[c, d, a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -216,7 +216,7 @@ MAX(DISTINCT a) FROM MyTable]]> ($1, 0))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -230,7 +230,7 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_SUM( +- LocalHashAggregate(groupBy=[a, $f1, $e], select=[a, $f1, $e, Partial_MAX(a_0) AS max$0]) +- Expand(projects=[{a, $f1, 0 AS $e, a AS a_0}, {a, null AS $f1, 1 AS $e, a AS a_0}, {null AS a, null AS $f1, 3 AS $e, a AS a_0}]) +- Calc(select=[a, (b > 0) IS TRUE AS $f1]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -243,7 +243,7 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_SUM( LogicalProject(EXPR$0=[$2], EXPR$1=[$3], EXPR$2=[$4]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}]], EXPR$0=[COUNT(DISTINCT $2)], EXPR$1=[SUM(DISTINCT $2)], EXPR$2=[MAX($2)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -257,7 +257,7 @@ Calc(select=[EXPR$0, EXPR$1, EXPR$2]) +- LocalHashAggregate(groupBy=[b, c, a, $e], select=[b, c, a, $e]) +- Expand(projects=[{b, null AS c, a, 1 AS $e}, {null AS b, c, a, 2 AS $e}]) +- Calc(select=[b, c, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -270,7 +270,7 @@ SUM(DISTINCT b), COUNT(c) FROM MyTable]]> ($2, 0))], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -285,7 +285,7 @@ Calc(select=[EXPR$0, EXPR$1, CASE(EXPR$2 IS NOT NULL, EXPR$2, 0) AS EXPR$2]) +- LocalHashAggregate(groupBy=[a, $f1, b, $e], select=[a, $f1, b, $e, Partial_COUNT(c) AS count$0]) +- Expand(projects=[{a, $f1, null AS b, c, 1 AS $e}, {null AS a, null AS $f1, b, c, 6 AS $e}, {null AS a, null AS $f1, null AS b, c, 7 AS $e}]) +- Calc(select=[a, (c > 0) IS TRUE AS $f1, b, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -298,7 +298,7 @@ Calc(select=[EXPR$0, EXPR$1, CASE(EXPR$2 IS NOT NULL, EXPR$2, 0) AS EXPR$2]) LogicalProject(EXPR$0=[$2], EXPR$1=[$3], EXPR$2=[$4]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}]], EXPR$0=[COUNT(DISTINCT $2)], EXPR$1=[SUM(DISTINCT $3)], EXPR$2=[COUNT($4)]) +- LogicalProject(d=[$3], e=[$4], a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -313,7 +313,7 @@ Calc(select=[EXPR$0, EXPR$1, CAST(EXPR$2 AS BIGINT) AS EXPR$2]) +- LocalHashAggregate(groupBy=[a, b, d, e, $e, $e_0], select=[a, b, d, e, $e, $e_0, Partial_COUNT(c) AS count$0]) +- Expand(projects=[{a, null AS b, c, d, e, $e, 8 AS $e_0}, {null AS a, b, c, d, e, $e, 16 AS $e_0}, {null AS a, null AS b, c, d, e, $e, 24 AS $e_0}]) +- Expand(projects=[{a, b, c, d, null AS e, 1 AS $e}, {a, b, c, null AS d, e, 2 AS $e}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -327,7 +327,7 @@ FROM MyTable2 GROUP BY d]]> ($0, 5))], b=[$1], $f6=[IS TRUE(>($1, 3))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -345,7 +345,7 @@ SortAggregate(isMerge=[true], groupBy=[d], select=[d, Final_MIN(min$0) AS EXPR$1 +- Exchange(distribution=[hash[d, c, $f4, b, $f6, $e]]) +- Expand(projects=[{d, e, $f2, c, $f4, null AS b, null AS $f6, 3 AS $e}, {d, e, $f2, c, null AS $f4, null AS b, null AS $f6, 7 AS $e}, {d, e, $f2, null AS c, null AS $f4, b, $f6, 12 AS $e}, {d, e, $f2, null AS c, null AS $f4, null AS b, null AS $f6, 15 AS $e}]) +- Calc(select=[d, e, (a < 10) IS TRUE AS $f2, c, (a > 5) IS TRUE AS $f4, b, (b > 3) IS TRUE AS $f6]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -358,7 +358,7 @@ COUNT(DISTINCT b) FILTER (WHERE b > 1) FROM MyTable2 GROUP BY d]]> ($0, 0))], b=[$1], $f4=[IS TRUE(>($1, 1))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -372,7 +372,7 @@ HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS EX +- LocalHashAggregate(groupBy=[d, c, $f2, b, $f4, $e], select=[d, c, $f2, b, $f4, $e]) +- Expand(projects=[{d, c, $f2, null AS b, null AS $f4, 3 AS $e}, {d, c, null AS $f2, null AS b, null AS $f4, 7 AS $e}, {d, null AS c, null AS $f2, b, $f4, 12 AS $e}]) +- Calc(select=[d, c, (a > 0) IS TRUE AS $f2, b, (b > 1) IS TRUE AS $f4]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -385,7 +385,7 @@ COUNT(DISTINCT c) FILTER (WHERE a < 10) FROM MyTable2 GROUP BY d]]> ($0, 10))], $f3=[IS TRUE(<($0, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -399,7 +399,7 @@ HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS EX +- LocalHashAggregate(groupBy=[d, c, $f2, $f3, $e], select=[d, c, $f2, $f3, $e]) +- Expand(projects=[{d, c, $f2, null AS $f3, 1 AS $e}, {d, c, null AS $f2, $f3, 2 AS $e}, {d, c, null AS $f2, null AS $f3, 3 AS $e}]) +- Calc(select=[d, c, (a > 10) IS TRUE AS $f2, (a < 10) IS TRUE AS $f3]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -412,7 +412,7 @@ MAX(e), MIN(e) FROM MyTable2 GROUP BY d]]> ($0, 0))], e=[$4]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -430,7 +430,7 @@ SortAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS EX +- Exchange(distribution=[hash[d, c, $f2, $e]]) +- Expand(projects=[{d, c, $f2, e, 0 AS $e}, {d, c, null AS $f2, e, 1 AS $e}, {d, null AS c, null AS $f2, e, 3 AS $e}]) +- Calc(select=[d, c, (a > 0) IS TRUE AS $f2, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -442,7 +442,7 @@ SortAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS EX @@ -454,7 +454,7 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0]) +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a]) +- Calc(select=[a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -466,7 +466,7 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0]) ($0, 0))], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -480,7 +480,7 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_MIN( +- LocalHashAggregate(groupBy=[a, $f1, $e], select=[a, $f1, $e, Partial_SUM(b) AS sum$0]) +- Expand(projects=[{a, $f1, b, 0 AS $e}, {null AS a, null AS $f1, b, 3 AS $e}]) +- Calc(select=[a, (a > 0) IS TRUE AS $f1, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -492,7 +492,7 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_MIN( ($0, 0))], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -507,7 +507,7 @@ Calc(select=[CASE(EXPR$0 IS NOT NULL, EXPR$0, 0) AS EXPR$0, EXPR$1]) +- LocalHashAggregate(groupBy=[b, $e], select=[b, $e, Partial_COUNT(a) FILTER $f1 AS count$0]) +- Expand(projects=[{a, $f1, b, 0 AS $e}, {a, $f1, null AS b, 1 AS $e}]) +- Calc(select=[a, (a > 0) IS TRUE AS $f1, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -520,7 +520,7 @@ Calc(select=[CASE(EXPR$0 IS NOT NULL, EXPR$0, 0) AS EXPR$0, EXPR$1]) LogicalProject(EXPR$0=[$2], EXPR$1=[$3]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}]], EXPR$0=[COUNT(DISTINCT $2)], EXPR$1=[SUM($0)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -536,7 +536,7 @@ Calc(select=[EXPR$0, EXPR$1]) +- Expand(projects=[{b, c, a, $e, b_0, 0 AS $e_0}, {b, c, null AS a, $e, b_0, 2 AS $e_0}]) +- Expand(projects=[{b, null AS c, a, 1 AS $e, b AS b_0}, {null AS b, c, a, 2 AS $e, b AS b_0}]) +- Calc(select=[b, c, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -549,7 +549,7 @@ Calc(select=[EXPR$0, EXPR$1]) LogicalProject(EXPR$0=[$2], EXPR$1=[$3]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}]], EXPR$0=[COUNT($2)], EXPR$1=[SUM(DISTINCT $3)]) +- LogicalProject(c=[$2], d=[$3], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -565,7 +565,7 @@ Calc(select=[CAST(EXPR$0 AS BIGINT) AS EXPR$0, EXPR$1]) +- Expand(projects=[{c, d, a, b, $e, 0 AS $e_0}, {c, d, a, null AS b, $e, 2 AS $e_0}]) +- Expand(projects=[{c, null AS d, a, b, 1 AS $e}, {null AS c, d, a, b, 2 AS $e}]) +- Calc(select=[c, d, a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -577,7 +577,7 @@ Calc(select=[CAST(EXPR$0 AS BIGINT) AS EXPR$0, EXPR$1]) @@ -592,7 +592,7 @@ Calc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2, EXPR$3]) +- LocalHashAggregate(groupBy=[a, b, $e], select=[a, b, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{a, b, 0 AS $e}, {a, null AS b, 1 AS $e}]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -604,7 +604,7 @@ Calc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2, EXPR$3]) @@ -619,7 +619,7 @@ Calc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2]) +- LocalHashAggregate(groupBy=[a, b, $e], select=[a, b, $e, Partial_COUNT(a) AS count$0]) +- Expand(projects=[{a, b, 0 AS $e}, {a, null AS b, 1 AS $e}]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -631,7 +631,7 @@ Calc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2]) @@ -646,7 +646,7 @@ Calc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2]) +- LocalHashAggregate(groupBy=[a, b, $e], select=[a, b, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{a, b, 0 AS $e}, {a, null AS b, 1 AS $e}]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -658,7 +658,7 @@ Calc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2]) ($0, 0))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -670,7 +670,7 @@ HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS EX +- Exchange(distribution=[hash[d, $f2, i$c]]) +- LocalHashAggregate(groupBy=[d, $f2, i$c], select=[d, $f2, i$c]) +- Calc(select=[d, (a > 0) IS TRUE AS $f2, CASE((a > 0) IS TRUE, c, null:VARCHAR(2147483647)) AS i$c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -681,7 +681,7 @@ HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS EX @@ -695,7 +695,7 @@ Calc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2, EXPR$3]) +- Exchange(distribution=[hash[a, b, c, $e]]) +- LocalHashAggregate(groupBy=[a, b, c, $e], select=[a, b, c, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/GroupWindowTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/GroupWindowTest.xml index 60c4e4af42188..843dd9acd8700 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/GroupWindowTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/GroupWindowTest.xml @@ -34,7 +34,7 @@ FROM MyTable1 LogicalProject(EXPR$0=[$1], EXPR$1=[$2], EXPR$2=[$3], EXPR$3=[$4], EXPR$4=[TUMBLE_START($0)], EXPR$5=[TUMBLE_END($0)]) +- LogicalAggregate(group=[{0}], EXPR$0=[VAR_POP($1)], EXPR$1=[VAR_SAMP($1)], EXPR$2=[STDDEV_POP($1)], EXPR$3=[STDDEV_SAMP($1)]) +- LogicalProject($f0=[$TUMBLE($0, 900000:INTERVAL MINUTE)], b=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -44,7 +44,7 @@ Calc(select=[CAST((($f0 - (($f1 * $f1) / $f2)) / $f2) AS INTEGER) AS EXPR$0, CAS +- Exchange(distribution=[single]) +- LocalHashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 900000)], properties=[w$start, w$end, w$rowtime], select=[Partial_SUM($f2) AS sum$0, Partial_SUM(b) AS sum$1, Partial_COUNT(b) AS count$2]) +- Calc(select=[ts, b, (b * b) AS $f2]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -66,7 +66,7 @@ FROM MyTable1 LogicalProject(EXPR$0=[$1], EXPR$1=[$2], EXPR$2=[$3], EXPR$3=[$4], EXPR$4=[TUMBLE_START($0)], EXPR$5=[TUMBLE_END($0)]) +- LogicalAggregate(group=[{0}], EXPR$0=[VAR_POP($1)], EXPR$1=[VAR_SAMP($1)], EXPR$2=[STDDEV_POP($1)], EXPR$3=[STDDEV_SAMP($1)]) +- LogicalProject($f0=[$TUMBLE($0, 900000:INTERVAL MINUTE)], b=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -75,7 +75,7 @@ Calc(select=[CAST((($f0 - (($f1 * $f1) / $f2)) / $f2) AS INTEGER) AS EXPR$0, CAS +- HashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 900000)], properties=[w$start, w$end, w$rowtime], select=[SUM($f2) AS $f0, SUM(b) AS $f1, COUNT(b) AS $f2]) +- Exchange(distribution=[single]) +- Calc(select=[ts, b, (b * b) AS $f2]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -97,7 +97,7 @@ FROM MyTable1 LogicalProject(EXPR$0=[$1], EXPR$1=[$2], EXPR$2=[$3], EXPR$3=[$4], EXPR$4=[TUMBLE_START($0)], EXPR$5=[TUMBLE_END($0)]) +- LogicalAggregate(group=[{0}], EXPR$0=[VAR_POP($1)], EXPR$1=[VAR_SAMP($1)], EXPR$2=[STDDEV_POP($1)], EXPR$3=[STDDEV_SAMP($1)]) +- LogicalProject($f0=[$TUMBLE($0, 900000:INTERVAL MINUTE)], b=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -107,7 +107,7 @@ Calc(select=[CAST((($f0 - (($f1 * $f1) / $f2)) / $f2) AS INTEGER) AS EXPR$0, CAS +- Exchange(distribution=[single]) +- LocalHashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 900000)], properties=[w$start, w$end, w$rowtime], select=[Partial_SUM($f2) AS sum$0, Partial_SUM(b) AS sum$1, Partial_COUNT(b) AS count$2]) +- Calc(select=[ts, b, (b * b) AS $f2]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -129,7 +129,7 @@ LogicalProject(EXPR$0=[$1], EXPR$1=[HOP_START($0)]) +- LogicalFilter(condition=[AND(>($2, 0), =(EXTRACT(FLAG(QUARTER), HOP_START($0)), 1))]) +- LogicalAggregate(group=[{0}], EXPR$0=[COUNT()], agg#1=[SUM($1)]) +- LogicalProject($f0=[$HOP($4, 900000:INTERVAL MINUTE, 60000:INTERVAL MINUTE)], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -139,7 +139,7 @@ Calc(select=[EXPR$0, w$start AS EXPR$1], where=[(($f1 > 0) AND (EXTRACT(QUARTER, +- Exchange(distribution=[single]) +- LocalHashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 60000, 900000)], properties=[w$start, w$end, w$rowtime], select=[Partial_COUNT(*) AS count1$0, Partial_SUM(a) AS sum$1]) +- Calc(select=[ts, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -161,7 +161,7 @@ LogicalProject(EXPR$0=[$1], EXPR$1=[HOP_START($0)]) +- LogicalFilter(condition=[AND(>($2, 0), =(EXTRACT(FLAG(QUARTER), HOP_START($0)), 1))]) +- LogicalAggregate(group=[{0}], EXPR$0=[COUNT()], agg#1=[SUM($1)]) +- LogicalProject($f0=[$HOP($4, 900000:INTERVAL MINUTE, 60000:INTERVAL MINUTE)], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -170,7 +170,7 @@ Calc(select=[EXPR$0, w$start AS EXPR$1], where=[(($f1 > 0) AND (EXTRACT(QUARTER, +- HashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 60000, 900000)], properties=[w$start, w$end, w$rowtime], select=[COUNT(*) AS EXPR$0, SUM(a) AS $f1]) +- Exchange(distribution=[single]) +- Calc(select=[ts, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -192,7 +192,7 @@ LogicalProject(EXPR$0=[$1], EXPR$1=[HOP_START($0)]) +- LogicalFilter(condition=[AND(>($2, 0), =(EXTRACT(FLAG(QUARTER), HOP_START($0)), 1))]) +- LogicalAggregate(group=[{0}], EXPR$0=[COUNT()], agg#1=[SUM($1)]) +- LogicalProject($f0=[$HOP($4, 900000:INTERVAL MINUTE, 60000:INTERVAL MINUTE)], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -202,7 +202,7 @@ Calc(select=[EXPR$0, w$start AS EXPR$1], where=[(($f1 > 0) AND (EXTRACT(QUARTER, +- Exchange(distribution=[single]) +- LocalHashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 60000, 900000)], properties=[w$start, w$end, w$rowtime], select=[Partial_COUNT(*) AS count1$0, Partial_SUM(a) AS sum$1]) +- Calc(select=[ts, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -215,7 +215,7 @@ Calc(select=[EXPR$0, w$start AS EXPR$1], where=[(($f1 > 0) AND (EXTRACT(QUARTER, LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[COUNT($2)]) +- LogicalProject(a=[$1], $f1=[$HOP($0, 3000:INTERVAL SECOND, 3600000:INTERVAL HOUR)], c=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -225,7 +225,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[hash[a]]) +- LocalHashWindowAggregate(groupBy=[a], window=[SlidingGroupWindow('w$, ts, 3600000, 3000)], select=[a, Partial_COUNT(c) AS count$0]) +- Calc(select=[a, ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -244,7 +244,7 @@ FROM MyTable LogicalProject(EXPR$0=[$1], EXPR$1=[HOP_START($0)], EXPR$2=[HOP_END($0)]) +- LogicalAggregate(group=[{0}], EXPR$0=[SUM($1)]) +- LogicalProject($f0=[$HOP($1, 3000:INTERVAL SECOND, 3000:INTERVAL SECOND)], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -254,7 +254,7 @@ Calc(select=[EXPR$0, w$start AS EXPR$1, w$end AS EXPR$2]) +- Exchange(distribution=[single]) +- LocalHashWindowAggregate(window=[SlidingGroupWindow('w$, b, 3000, 3000)], properties=[w$start, w$end, w$rowtime], select=[Partial_SUM(a) AS sum$0]) +- Calc(select=[b, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -273,7 +273,7 @@ FROM MyTable LogicalProject(EXPR$0=[$1], EXPR$1=[HOP_START($0)], EXPR$2=[HOP_END($0)]) +- LogicalAggregate(group=[{0}], EXPR$0=[SUM($1)]) +- LogicalProject($f0=[$HOP($1, 3000:INTERVAL SECOND, 3000:INTERVAL SECOND)], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -282,7 +282,7 @@ Calc(select=[EXPR$0, w$start AS EXPR$1, w$end AS EXPR$2]) +- HashWindowAggregate(window=[SlidingGroupWindow('w$, b, 3000, 3000)], properties=[w$start, w$end, w$rowtime], select=[SUM(a) AS EXPR$0]) +- Exchange(distribution=[single]) +- Calc(select=[b, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -301,7 +301,7 @@ FROM MyTable LogicalProject(EXPR$0=[$1], EXPR$1=[HOP_START($0)], EXPR$2=[HOP_END($0)]) +- LogicalAggregate(group=[{0}], EXPR$0=[SUM($1)]) +- LogicalProject($f0=[$HOP($1, 3000:INTERVAL SECOND, 3000:INTERVAL SECOND)], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -311,7 +311,7 @@ Calc(select=[EXPR$0, w$start AS EXPR$1, w$end AS EXPR$2]) +- Exchange(distribution=[single]) +- LocalHashWindowAggregate(window=[SlidingGroupWindow('w$, b, 3000, 3000)], properties=[w$start, w$end, w$rowtime], select=[Partial_SUM(a) AS sum$0]) +- Calc(select=[b, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -324,7 +324,7 @@ Calc(select=[EXPR$0, w$start AS EXPR$1, w$end AS EXPR$2]) LogicalProject(EXPR$0=[$1], EXPR$1=[$2]) +- LogicalAggregate(group=[{0}], EXPR$0=[AVG($1)], EXPR$1=[SUM($2)]) +- LogicalProject($f0=[$TUMBLE($1, 3000:INTERVAL SECOND)], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -333,7 +333,7 @@ HashWindowAggregate(window=[TumblingGroupWindow('w$, b, 3000)], select=[Final_AV +- Exchange(distribution=[single]) +- LocalHashWindowAggregate(window=[TumblingGroupWindow('w$, b, 3000)], select=[Partial_AVG(c) AS (sum$0, count$1), Partial_SUM(a) AS sum$2]) +- Calc(select=[b, c, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -346,7 +346,7 @@ HashWindowAggregate(window=[TumblingGroupWindow('w$, b, 3000)], select=[Final_AV LogicalProject(EXPR$0=[$1], EXPR$1=[$2]) +- LogicalAggregate(group=[{0}], EXPR$0=[AVG($1)], EXPR$1=[SUM($2)]) +- LogicalProject($f0=[$TUMBLE($1, 3000:INTERVAL SECOND)], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -354,7 +354,7 @@ LogicalProject(EXPR$0=[$1], EXPR$1=[$2]) HashWindowAggregate(window=[TumblingGroupWindow('w$, b, 3000)], select=[AVG(c) AS EXPR$0, SUM(a) AS EXPR$1]) +- Exchange(distribution=[single]) +- Calc(select=[b, c, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -367,7 +367,7 @@ HashWindowAggregate(window=[TumblingGroupWindow('w$, b, 3000)], select=[AVG(c) A LogicalProject(EXPR$0=[$1], EXPR$1=[$2]) +- LogicalAggregate(group=[{0}], EXPR$0=[AVG($1)], EXPR$1=[SUM($2)]) +- LogicalProject($f0=[$TUMBLE($1, 3000:INTERVAL SECOND)], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -376,7 +376,7 @@ HashWindowAggregate(window=[TumblingGroupWindow('w$, b, 3000)], select=[Final_AV +- Exchange(distribution=[single]) +- LocalHashWindowAggregate(window=[TumblingGroupWindow('w$, b, 3000)], select=[Partial_AVG(c) AS (sum$0, count$1), Partial_SUM(a) AS sum$2]) +- Calc(select=[b, c, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -389,7 +389,7 @@ HashWindowAggregate(window=[TumblingGroupWindow('w$, b, 3000)], select=[Final_AV LogicalProject(sumA=[$1], cntB=[$2]) +- LogicalAggregate(group=[{0}], sumA=[SUM($1)], cntB=[COUNT($2)]) +- LogicalProject($f0=[$HOP($4, 900000:INTERVAL MINUTE, 5400000:INTERVAL MINUTE)], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -398,7 +398,7 @@ SortWindowAggregate(window=[SlidingGroupWindow('w$, ts, 5400000, 900000)], selec +- Sort(orderBy=[ts ASC]) +- Exchange(distribution=[single]) +- Calc(select=[ts, a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -411,7 +411,7 @@ SortWindowAggregate(window=[SlidingGroupWindow('w$, ts, 5400000, 900000)], selec LogicalProject(sumA=[$1], cntB=[$2]) +- LogicalAggregate(group=[{0}], sumA=[SUM($1)], cntB=[COUNT($2)]) +- LogicalProject($f0=[$HOP($4, 900000:INTERVAL MINUTE, 5400000:INTERVAL MINUTE)], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -420,7 +420,7 @@ HashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 5400000, 900000)], selec +- Exchange(distribution=[single]) +- LocalHashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 5400000, 900000)], select=[Partial_SUM(a) AS sum$0, Partial_COUNT(b) AS count$1]) +- Calc(select=[ts, a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -433,7 +433,7 @@ HashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 5400000, 900000)], selec LogicalProject(sumA=[$1], cntB=[$2]) +- LogicalAggregate(group=[{0}], sumA=[SUM($1)], cntB=[COUNT($2)]) +- LogicalProject($f0=[$TUMBLE($4, 7200000:INTERVAL HOUR)], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -442,7 +442,7 @@ HashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 7200000)], select=[Fina +- Exchange(distribution=[single]) +- LocalHashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 7200000)], select=[Partial_SUM(a) AS sum$0, Partial_COUNT(b) AS count$1]) +- Calc(select=[ts, a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -455,7 +455,7 @@ HashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 7200000)], select=[Fina LogicalProject(sumA=[$1], cntB=[$2]) +- LogicalAggregate(group=[{0}], sumA=[SUM($1)], cntB=[COUNT($2)]) +- LogicalProject($f0=[$TUMBLE($4, 7200000:INTERVAL HOUR)], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -463,7 +463,7 @@ LogicalProject(sumA=[$1], cntB=[$2]) HashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 7200000)], select=[SUM(a) AS sumA, COUNT(b) AS cntB]) +- Exchange(distribution=[single]) +- Calc(select=[ts, a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -476,7 +476,7 @@ HashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 7200000)], select=[SUM( LogicalProject(sumA=[$1], cntB=[$2]) +- LogicalAggregate(group=[{0}], sumA=[SUM($1)], cntB=[COUNT($2)]) +- LogicalProject($f0=[$TUMBLE($4, 7200000:INTERVAL HOUR)], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -485,7 +485,7 @@ HashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 7200000)], select=[Fina +- Exchange(distribution=[single]) +- LocalHashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 7200000)], select=[Partial_SUM(a) AS sum$0, Partial_COUNT(b) AS count$1]) +- Calc(select=[ts, a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -498,7 +498,7 @@ HashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 7200000)], select=[Fina LogicalProject(c=[$2], EXPR$1=[HOP_END($0)], EXPR$2=[HOP_START($0)], EXPR$3=[HOP_ROWTIME($0)], sumA=[$3], avgB=[$4]) +- LogicalAggregate(group=[{0, 1, 2}], sumA=[SUM($3)], avgB=[AVG($4)]) +- LogicalProject($f0=[$HOP($4, 3600000:INTERVAL HOUR, 10800000:INTERVAL HOUR)], d=[$3], c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -507,7 +507,7 @@ Calc(select=[c, w$end AS EXPR$1, w$start AS EXPR$2, w$rowtime AS EXPR$3, sumA, a +- HashWindowAggregate(groupBy=[c, d], window=[SlidingGroupWindow('w$, ts, 10800000, 3600000)], properties=[w$start, w$end, w$rowtime], select=[c, d, Final_SUM(sum$0) AS sumA, Final_AVG(sum$1, count$2) AS avgB]) +- Exchange(distribution=[hash[c, d]]) +- LocalHashWindowAggregate(groupBy=[c, d], window=[SlidingGroupWindow('w$, ts, 10800000, 3600000)], properties=[w$start, w$end, w$rowtime], select=[c, d, Partial_SUM(a) AS sum$0, Partial_AVG(b) AS (sum$1, count$2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -520,7 +520,7 @@ Calc(select=[c, w$end AS EXPR$1, w$start AS EXPR$2, w$rowtime AS EXPR$3, sumA, a LogicalProject(sumA=[$1], cntB=[$2]) +- LogicalAggregate(group=[{0}], sumA=[SUM($1)], cntB=[COUNT($2)]) +- LogicalProject($f0=[$HOP($4, 900000:INTERVAL MINUTE, 5400000:INTERVAL MINUTE)], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -529,7 +529,7 @@ HashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 5400000, 900000)], selec +- Exchange(distribution=[single]) +- LocalHashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 5400000, 900000)], select=[Partial_SUM(a) AS sum$0, Partial_COUNT(b) AS count$1]) +- Calc(select=[ts, a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -542,7 +542,7 @@ HashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 5400000, 900000)], selec LogicalProject(c=[$2], EXPR$1=[HOP_END($0)], EXPR$2=[HOP_START($0)], EXPR$3=[HOP_ROWTIME($0)], sumA=[$3], avgB=[$4]) +- LogicalAggregate(group=[{0, 1, 2}], sumA=[SUM($3)], avgB=[AVG($4)]) +- LogicalProject($f0=[$HOP($4, 3600000:INTERVAL HOUR, 10800000:INTERVAL HOUR)], d=[$3], c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -552,7 +552,7 @@ Calc(select=[c, w$end AS EXPR$1, w$start AS EXPR$2, w$rowtime AS EXPR$3, sumA, a +- Exchange(distribution=[forward]) +- Sort(orderBy=[c ASC, d ASC, ts ASC]) +- Exchange(distribution=[hash[c, d]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -565,7 +565,7 @@ Calc(select=[c, w$end AS EXPR$1, w$start AS EXPR$2, w$rowtime AS EXPR$3, sumA, a LogicalProject(c=[$2], EXPR$1=[HOP_END($0)], EXPR$2=[HOP_START($0)], EXPR$3=[HOP_ROWTIME($0)], sumA=[$3], avgB=[$4]) +- LogicalAggregate(group=[{0, 1, 2}], sumA=[SUM($3)], avgB=[AVG($4)]) +- LogicalProject($f0=[$HOP($4, 3600000:INTERVAL HOUR, 10800000:INTERVAL HOUR)], d=[$3], c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -574,7 +574,7 @@ Calc(select=[c, w$end AS EXPR$1, w$start AS EXPR$2, w$rowtime AS EXPR$3, sumA, a +- HashWindowAggregate(groupBy=[c, d], window=[SlidingGroupWindow('w$, ts, 10800000, 3600000)], properties=[w$start, w$end, w$rowtime], select=[c, d, Final_SUM(sum$0) AS sumA, Final_AVG(sum$1, count$2) AS avgB]) +- Exchange(distribution=[hash[c, d]]) +- LocalHashWindowAggregate(groupBy=[c, d], window=[SlidingGroupWindow('w$, ts, 10800000, 3600000)], properties=[w$start, w$end, w$rowtime], select=[c, d, Partial_SUM(a) AS sum$0, Partial_AVG(b) AS (sum$1, count$2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -596,7 +596,7 @@ FROM MyTable2 LogicalProject(EXPR$0=[TUMBLE_START($0)], EXPR$1=[TUMBLE_END($0)], EXPR$2=[TUMBLE_ROWTIME($0)], c=[$1], sumA=[$2], minB=[$3]) +- LogicalAggregate(group=[{0, 1}], sumA=[SUM($2)], minB=[MIN($3)]) +- LogicalProject($f0=[$TUMBLE($4, 240000:INTERVAL MINUTE)], c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -606,7 +606,7 @@ Calc(select=[w$start AS EXPR$0, w$end AS EXPR$1, w$rowtime AS EXPR$2, c, sumA, m +- Exchange(distribution=[hash[c]]) +- LocalHashWindowAggregate(groupBy=[c], window=[TumblingGroupWindow('w$, ts, 240000)], properties=[w$start, w$end, w$rowtime], select=[c, Partial_SUM(a) AS sum$0, Partial_MIN(b) AS min$1]) +- Calc(select=[ts, c, a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -628,7 +628,7 @@ FROM MyTable2 LogicalProject(EXPR$0=[TUMBLE_START($0)], EXPR$1=[TUMBLE_END($0)], EXPR$2=[TUMBLE_ROWTIME($0)], c=[$1], sumA=[$2], minB=[$3]) +- LogicalAggregate(group=[{0, 1}], sumA=[SUM($2)], minB=[MIN($3)]) +- LogicalProject($f0=[$TUMBLE($4, 240000:INTERVAL MINUTE)], c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -637,7 +637,7 @@ Calc(select=[w$start AS EXPR$0, w$end AS EXPR$1, w$rowtime AS EXPR$2, c, sumA, m +- HashWindowAggregate(groupBy=[c], window=[TumblingGroupWindow('w$, ts, 240000)], properties=[w$start, w$end, w$rowtime], select=[c, SUM(a) AS sumA, MIN(b) AS minB]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[ts, c, a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -659,7 +659,7 @@ FROM MyTable2 LogicalProject(EXPR$0=[TUMBLE_START($0)], EXPR$1=[TUMBLE_END($0)], EXPR$2=[TUMBLE_ROWTIME($0)], c=[$1], sumA=[$2], minB=[$3]) +- LogicalAggregate(group=[{0, 1}], sumA=[SUM($2)], minB=[MIN($3)]) +- LogicalProject($f0=[$TUMBLE($4, 240000:INTERVAL MINUTE)], c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -669,7 +669,7 @@ Calc(select=[w$start AS EXPR$0, w$end AS EXPR$1, w$rowtime AS EXPR$2, c, sumA, m +- Exchange(distribution=[hash[c]]) +- LocalHashWindowAggregate(groupBy=[c], window=[TumblingGroupWindow('w$, ts, 240000)], properties=[w$start, w$end, w$rowtime], select=[c, Partial_SUM(a) AS sum$0, Partial_MIN(b) AS min$1]) +- Calc(select=[ts, c, a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -696,7 +696,7 @@ LogicalProject(s=[$1], a=[$2], wStart=[TUMBLE_START($0)]) +- LogicalAggregate(group=[{0}], s=[SUM($1)], a=[AVG($1)]) +- LogicalProject($f0=[$TUMBLE($1, 900000:INTERVAL MINUTE)], correct=[$0]) +- LogicalProject(correct=[CASE(=($0, 1), 1, 99)], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -706,7 +706,7 @@ Calc(select=[CAST(s AS INTEGER) AS s, CAST(a AS INTEGER) AS a, w$start AS wStart +- Exchange(distribution=[single]) +- LocalHashWindowAggregate(window=[TumblingGroupWindow('w$, b, 900000)], properties=[w$start, w$end, w$rowtime], select=[Partial_SUM(correct) AS sum$0, Partial_AVG(correct) AS (sum$1, count$2)]) +- Calc(select=[b, CASE((a = 1), 1, 99) AS correct]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -733,7 +733,7 @@ LogicalProject(s=[$1], a=[$2], wStart=[TUMBLE_START($0)]) +- LogicalAggregate(group=[{0}], s=[SUM($1)], a=[AVG($1)]) +- LogicalProject($f0=[$TUMBLE($1, 900000:INTERVAL MINUTE)], correct=[$0]) +- LogicalProject(correct=[CASE(=($0, 1), 1, 99)], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -742,7 +742,7 @@ Calc(select=[CAST(s AS INTEGER) AS s, CAST(a AS INTEGER) AS a, w$start AS wStart +- HashWindowAggregate(window=[TumblingGroupWindow('w$, b, 900000)], properties=[w$start, w$end, w$rowtime], select=[SUM(correct) AS s, AVG(correct) AS a]) +- Exchange(distribution=[single]) +- Calc(select=[b, CASE((a = 1), 1, 99) AS correct]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -769,7 +769,7 @@ LogicalProject(s=[$1], a=[$2], wStart=[TUMBLE_START($0)]) +- LogicalAggregate(group=[{0}], s=[SUM($1)], a=[AVG($1)]) +- LogicalProject($f0=[$TUMBLE($1, 900000:INTERVAL MINUTE)], correct=[$0]) +- LogicalProject(correct=[CASE(=($0, 1), 1, 99)], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -779,7 +779,7 @@ Calc(select=[CAST(s AS INTEGER) AS s, CAST(a AS INTEGER) AS a, w$start AS wStart +- Exchange(distribution=[single]) +- LocalHashWindowAggregate(window=[TumblingGroupWindow('w$, b, 900000)], properties=[w$start, w$end, w$rowtime], select=[Partial_SUM(correct) AS sum$0, Partial_AVG(correct) AS (sum$1, count$2)]) +- Calc(select=[b, CASE((a = 1), 1, 99) AS correct]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -792,7 +792,7 @@ Calc(select=[CAST(s AS INTEGER) AS s, CAST(a AS INTEGER) AS a, w$start AS wStart LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[COUNT($2)]) +- LogicalProject(b=[$2], $f1=[$HOP($0, 3000:INTERVAL SECOND, 3600000:INTERVAL HOUR)], c=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -802,7 +802,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[hash[b]]) +- LocalHashWindowAggregate(groupBy=[b], window=[SlidingGroupWindow('w$, ts, 3600000, 3000)], select=[b, Partial_COUNT(c) AS count$0]) +- Calc(select=[b, ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -815,7 +815,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[COUNT($2)]) +- LogicalProject(b=[$2], $f1=[$HOP($0, 3000:INTERVAL SECOND, 3600000:INTERVAL HOUR)], c=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -826,7 +826,7 @@ Calc(select=[EXPR$0]) +- Sort(orderBy=[b ASC, ts ASC]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[b, ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -839,7 +839,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[COUNT($2)]) +- LogicalProject(b=[$2], $f1=[$HOP($0, 3000:INTERVAL SECOND, 3600000:INTERVAL HOUR)], c=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -849,7 +849,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[hash[b]]) +- LocalHashWindowAggregate(groupBy=[b], window=[SlidingGroupWindow('w$, ts, 3600000, 3000)], select=[b, Partial_COUNT(c) AS count$0]) +- Calc(select=[b, ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -862,7 +862,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[MAX($2)]) +- LogicalProject(a=[$1], $f1=[$HOP($0, 3000:INTERVAL SECOND, 3600000:INTERVAL HOUR)], c=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -876,7 +876,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC, ts ASC]) +- Calc(select=[a, ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -889,7 +889,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[MAX($2)]) +- LogicalProject(a=[$1], $f1=[$HOP($0, 3000:INTERVAL SECOND, 3600000:INTERVAL HOUR)], c=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -900,7 +900,7 @@ Calc(select=[EXPR$0]) +- Sort(orderBy=[a ASC, ts ASC]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -913,7 +913,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[MAX($2)]) +- LogicalProject(a=[$1], $f1=[$HOP($0, 3000:INTERVAL SECOND, 3600000:INTERVAL HOUR)], c=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -927,7 +927,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC, ts ASC]) +- Calc(select=[a, ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -940,7 +940,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[MAX($2)]) +- LogicalProject(b=[$2], $f1=[$HOP($0, 111:INTERVAL SECOND(1, 3), 1000:INTERVAL SECOND)], c=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -951,7 +951,7 @@ Calc(select=[EXPR$0]) +- Sort(orderBy=[b ASC, ts ASC]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[b, ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -964,7 +964,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[MAX($2)]) +- LogicalProject(b=[$2], $f1=[$HOP($0, 111:INTERVAL SECOND(1, 3), 1000:INTERVAL SECOND)], c=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -975,7 +975,7 @@ Calc(select=[EXPR$0]) +- Sort(orderBy=[b ASC, ts ASC]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[b, ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -988,7 +988,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[MAX($2)]) +- LogicalProject(b=[$2], $f1=[$HOP($0, 111:INTERVAL SECOND(1, 3), 1000:INTERVAL SECOND)], c=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -999,7 +999,7 @@ Calc(select=[EXPR$0]) +- Sort(orderBy=[b ASC, ts ASC]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[b, ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -1012,7 +1012,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$0=[countFun($3)]) +- LogicalProject(a=[$0], d=[$3], $f2=[$HOP($1, 3000:INTERVAL SECOND, 3600000:INTERVAL HOUR)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1022,7 +1022,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC, d ASC, b ASC]) +- Exchange(distribution=[hash[a, d]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -1035,7 +1035,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$0=[countFun($3)]) +- LogicalProject(a=[$0], d=[$3], $f2=[$HOP($1, 3000:INTERVAL SECOND, 3600000:INTERVAL HOUR)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1045,7 +1045,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC, d ASC, b ASC]) +- Exchange(distribution=[hash[a, d]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -1058,7 +1058,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$0=[countFun($3)]) +- LogicalProject(a=[$0], d=[$3], $f2=[$HOP($1, 3000:INTERVAL SECOND, 3600000:INTERVAL HOUR)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1071,7 +1071,7 @@ Calc(select=[EXPR$0]) +- LocalSortWindowAggregate(groupBy=[a, d], window=[SlidingGroupWindow('w$, b, 3600000, 3000)], select=[a, d, Partial_countFun(c) AS EXPR$0]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC, d ASC, b ASC]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -1084,7 +1084,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[COUNT($2)]) +- LogicalProject(a=[$1], $f1=[$HOP($0, 3000:INTERVAL SECOND, 3600000:INTERVAL HOUR)], c=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -1095,7 +1095,7 @@ Calc(select=[EXPR$0]) +- Sort(orderBy=[a ASC, ts ASC]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -1108,7 +1108,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[COUNT($2)]) +- LogicalProject(a=[$1], $f1=[$HOP($0, 3000:INTERVAL SECOND, 3600000:INTERVAL HOUR)], c=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -1118,7 +1118,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[hash[a]]) +- LocalHashWindowAggregate(groupBy=[a], window=[SlidingGroupWindow('w$, ts, 3600000, 3000)], select=[a, Partial_COUNT(c) AS count$0]) +- Calc(select=[a, ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -1131,7 +1131,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[COUNT($2)]) +- LogicalProject(a=[$1], $f1=[$TUMBLE($0, 3000:INTERVAL SECOND)], c=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -1141,7 +1141,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[hash[a]]) +- LocalHashWindowAggregate(groupBy=[a], window=[TumblingGroupWindow('w$, ts, 3000)], select=[a, Partial_COUNT(c) AS count$0]) +- Calc(select=[a, ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -1154,7 +1154,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[COUNT($2)]) +- LogicalProject(a=[$1], $f1=[$TUMBLE($0, 3000:INTERVAL SECOND)], c=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -1163,7 +1163,7 @@ Calc(select=[EXPR$0]) +- HashWindowAggregate(groupBy=[a], window=[TumblingGroupWindow('w$, ts, 3000)], select=[a, COUNT(c) AS EXPR$0]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -1176,7 +1176,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[COUNT($2)]) +- LogicalProject(a=[$1], $f1=[$TUMBLE($0, 3000:INTERVAL SECOND)], c=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -1186,7 +1186,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[hash[a]]) +- LocalHashWindowAggregate(groupBy=[a], window=[TumblingGroupWindow('w$, ts, 3000)], select=[a, Partial_COUNT(c) AS count$0]) +- Calc(select=[a, ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -1199,7 +1199,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$3], EXPR$1=[$4]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$0=[AVG($3)], EXPR$1=[COUNT($0)]) +- LogicalProject(a=[$0], d=[$3], $f2=[$TUMBLE($1, 3000:INTERVAL SECOND)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1208,7 +1208,7 @@ Calc(select=[EXPR$0, EXPR$1]) +- HashWindowAggregate(groupBy=[a, d], window=[TumblingGroupWindow('w$, b, 3000)], select=[a, d, Final_AVG(sum$0, count$1) AS EXPR$0, Final_COUNT(count$2) AS EXPR$1]) +- Exchange(distribution=[hash[a, d]]) +- LocalHashWindowAggregate(groupBy=[a, d], window=[TumblingGroupWindow('w$, b, 3000)], select=[a, d, Partial_AVG(c) AS (sum$0, count$1), Partial_COUNT(a) AS count$2]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -1221,7 +1221,7 @@ Calc(select=[EXPR$0, EXPR$1]) LogicalProject(EXPR$0=[$3], EXPR$1=[$4]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$0=[AVG($3)], EXPR$1=[COUNT($0)]) +- LogicalProject(a=[$0], d=[$3], $f2=[$TUMBLE($1, 3000:INTERVAL SECOND)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1229,7 +1229,7 @@ LogicalProject(EXPR$0=[$3], EXPR$1=[$4]) Calc(select=[EXPR$0, EXPR$1]) +- HashWindowAggregate(groupBy=[a, d], window=[TumblingGroupWindow('w$, b, 3000)], select=[a, d, AVG(c) AS EXPR$0, COUNT(a) AS EXPR$1]) +- Exchange(distribution=[hash[a, d]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -1242,7 +1242,7 @@ Calc(select=[EXPR$0, EXPR$1]) LogicalProject(EXPR$0=[$3], EXPR$1=[$4]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$0=[AVG($3)], EXPR$1=[COUNT($0)]) +- LogicalProject(a=[$0], d=[$3], $f2=[$TUMBLE($1, 3000:INTERVAL SECOND)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1251,7 +1251,7 @@ Calc(select=[EXPR$0, EXPR$1]) +- HashWindowAggregate(groupBy=[a, d], window=[TumblingGroupWindow('w$, b, 3000)], select=[a, d, Final_AVG(sum$0, count$1) AS EXPR$0, Final_COUNT(count$2) AS EXPR$1]) +- Exchange(distribution=[hash[a, d]]) +- LocalHashWindowAggregate(groupBy=[a, d], window=[TumblingGroupWindow('w$, b, 3000)], select=[a, d, Partial_AVG(c) AS (sum$0, count$1), Partial_COUNT(a) AS count$2]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -1264,7 +1264,7 @@ Calc(select=[EXPR$0, EXPR$1]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[MAX($2)]) +- LogicalProject(a=[$1], $f1=[$TUMBLE($0, 3000:INTERVAL SECOND)], c=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -1278,7 +1278,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC, ts ASC]) +- Calc(select=[a, ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -1291,7 +1291,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[MAX($2)]) +- LogicalProject(a=[$1], $f1=[$TUMBLE($0, 3000:INTERVAL SECOND)], c=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -1302,7 +1302,7 @@ Calc(select=[EXPR$0]) +- Sort(orderBy=[a ASC, ts ASC]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -1315,7 +1315,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[MAX($2)]) +- LogicalProject(a=[$1], $f1=[$TUMBLE($0, 3000:INTERVAL SECOND)], c=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -1329,7 +1329,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC, ts ASC]) +- Calc(select=[a, ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[ts, a, b, c]) ]]> @@ -1342,7 +1342,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$3], EXPR$1=[$4]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$0=[AVG($3)], EXPR$1=[countFun($0)]) +- LogicalProject(a=[$0], d=[$3], $f2=[$TUMBLE($1, 3000:INTERVAL SECOND)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1352,7 +1352,7 @@ Calc(select=[EXPR$0, EXPR$1]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC, d ASC, b ASC]) +- Exchange(distribution=[hash[a, d]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -1365,7 +1365,7 @@ Calc(select=[EXPR$0, EXPR$1]) LogicalProject(EXPR$0=[$3], EXPR$1=[$4]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$0=[AVG($3)], EXPR$1=[countFun($0)]) +- LogicalProject(a=[$0], d=[$3], $f2=[$TUMBLE($1, 3000:INTERVAL SECOND)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1375,7 +1375,7 @@ Calc(select=[EXPR$0, EXPR$1]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC, d ASC, b ASC]) +- Exchange(distribution=[hash[a, d]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -1388,7 +1388,7 @@ Calc(select=[EXPR$0, EXPR$1]) LogicalProject(EXPR$0=[$3], EXPR$1=[$4]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$0=[AVG($3)], EXPR$1=[countFun($0)]) +- LogicalProject(a=[$0], d=[$3], $f2=[$TUMBLE($1, 3000:INTERVAL SECOND)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1401,7 +1401,7 @@ Calc(select=[EXPR$0, EXPR$1]) +- LocalSortWindowAggregate(groupBy=[a, d], window=[TumblingGroupWindow('w$, b, 3000)], select=[a, d, Partial_AVG(c) AS (sum$0, count$1), Partial_countFun(a) AS EXPR$1]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC, d ASC, b ASC]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -1414,7 +1414,7 @@ Calc(select=[EXPR$0, EXPR$1]) LogicalProject(wAvg=[$1]) +- LogicalAggregate(group=[{0}], wAvg=[weightedAvg($1, $2)]) +- LogicalProject($f0=[$TUMBLE($4, 240000:INTERVAL MINUTE)], b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -1426,7 +1426,7 @@ SortWindowAggregate(window=[TumblingGroupWindow('w$, ts, 240000)], select=[Final +- Exchange(distribution=[forward]) +- Sort(orderBy=[ts ASC]) +- Calc(select=[ts, b, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -1439,7 +1439,7 @@ SortWindowAggregate(window=[TumblingGroupWindow('w$, ts, 240000)], select=[Final LogicalProject(wAvg=[$1]) +- LogicalAggregate(group=[{0}], wAvg=[weightedAvg($1, $2)]) +- LogicalProject($f0=[$TUMBLE($4, 240000:INTERVAL MINUTE)], b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -1448,7 +1448,7 @@ SortWindowAggregate(window=[TumblingGroupWindow('w$, ts, 240000)], select=[weigh +- Sort(orderBy=[ts ASC]) +- Exchange(distribution=[single]) +- Calc(select=[ts, b, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -1461,7 +1461,7 @@ SortWindowAggregate(window=[TumblingGroupWindow('w$, ts, 240000)], select=[weigh LogicalProject(wAvg=[$1]) +- LogicalAggregate(group=[{0}], wAvg=[weightedAvg($1, $2)]) +- LogicalProject($f0=[$TUMBLE($4, 240000:INTERVAL MINUTE)], b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -1473,7 +1473,7 @@ SortWindowAggregate(window=[TumblingGroupWindow('w$, ts, 240000)], select=[Final +- Exchange(distribution=[forward]) +- Sort(orderBy=[ts ASC]) +- Calc(select=[ts, b, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -1504,12 +1504,12 @@ LogicalUnion(all=[true]) : +- LogicalProject(EXPR$0=[1]) : +- LogicalAggregate(group=[{0}]) : +- LogicalProject($f0=[$HOP($4, 3600000:INTERVAL HOUR, 3600000:INTERVAL HOUR)]) -: +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) +- LogicalProject(EXPR$0=[$0]) +- LogicalProject(EXPR$0=[1]) +- LogicalAggregate(group=[{0}]) +- LogicalProject($f0=[$HOP($4, 3600000:INTERVAL HOUR, 7200000:INTERVAL HOUR)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -1520,7 +1520,7 @@ Union(all=[true], union=[EXPR$0]) : +- Exchange(distribution=[single]) : +- LocalHashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 3600000, 3600000)], select=[]) : +- Calc(select=[ts])(reuse_id=[1]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) +- Calc(select=[1 AS EXPR$0]) +- HashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 7200000, 3600000)], select=[]) +- Exchange(distribution=[single]) @@ -1538,7 +1538,7 @@ Union(all=[true], union=[EXPR$0]) LogicalProject(EXPR$0=[TUMBLE_END($0)]) +- LogicalAggregate(group=[{0, 1}]) +- LogicalProject($f0=[$TUMBLE($4, 240000:INTERVAL MINUTE)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -1548,7 +1548,7 @@ Calc(select=[w$end AS EXPR$0]) +- Exchange(distribution=[hash[c]]) +- LocalHashWindowAggregate(groupBy=[c], window=[TumblingGroupWindow('w$, ts, 240000)], properties=[w$start, w$end, w$rowtime], select=[c]) +- Calc(select=[ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -1579,12 +1579,12 @@ LogicalUnion(all=[true]) : +- LogicalProject(EXPR$0=[1]) : +- LogicalAggregate(group=[{0}]) : +- LogicalProject($f0=[$HOP($4, 3600000:INTERVAL HOUR, 3600000:INTERVAL HOUR)]) -: +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) +- LogicalProject(EXPR$0=[$0]) +- LogicalProject(EXPR$0=[1]) +- LogicalAggregate(group=[{0}]) +- LogicalProject($f0=[$HOP($4, 3600000:INTERVAL HOUR, 7200000:INTERVAL HOUR)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -1594,7 +1594,7 @@ Union(all=[true], union=[EXPR$0]) : +- HashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 3600000, 3600000)], select=[]) : +- Exchange(distribution=[single])(reuse_id=[1]) : +- Calc(select=[ts]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) +- Calc(select=[1 AS EXPR$0]) +- SortWindowAggregate(window=[SlidingGroupWindow('w$, ts, 7200000, 3600000)], select=[]) +- Sort(orderBy=[ts ASC]) @@ -1611,7 +1611,7 @@ Union(all=[true], union=[EXPR$0]) LogicalProject(EXPR$0=[TUMBLE_END($0)]) +- LogicalAggregate(group=[{0, 1}]) +- LogicalProject($f0=[$TUMBLE($4, 240000:INTERVAL MINUTE)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -1620,7 +1620,7 @@ Calc(select=[w$end AS EXPR$0]) +- HashWindowAggregate(groupBy=[c], window=[TumblingGroupWindow('w$, ts, 240000)], properties=[w$start, w$end, w$rowtime], select=[c]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> @@ -1651,12 +1651,12 @@ LogicalUnion(all=[true]) : +- LogicalProject(EXPR$0=[1]) : +- LogicalAggregate(group=[{0}]) : +- LogicalProject($f0=[$HOP($4, 3600000:INTERVAL HOUR, 3600000:INTERVAL HOUR)]) -: +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) +- LogicalProject(EXPR$0=[$0]) +- LogicalProject(EXPR$0=[1]) +- LogicalAggregate(group=[{0}]) +- LogicalProject($f0=[$HOP($4, 3600000:INTERVAL HOUR, 7200000:INTERVAL HOUR)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -1667,7 +1667,7 @@ Union(all=[true], union=[EXPR$0]) : +- Exchange(distribution=[single]) : +- LocalHashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 3600000, 3600000)], select=[]) : +- Calc(select=[ts])(reuse_id=[1]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) +- Calc(select=[1 AS EXPR$0]) +- HashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 7200000, 3600000)], select=[]) +- Exchange(distribution=[single]) @@ -1685,7 +1685,7 @@ Union(all=[true], union=[EXPR$0]) LogicalProject(EXPR$0=[TUMBLE_END($0)]) +- LogicalAggregate(group=[{0, 1}]) +- LogicalProject($f0=[$TUMBLE($4, 240000:INTERVAL MINUTE)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -1695,7 +1695,7 @@ Calc(select=[w$end AS EXPR$0]) +- Exchange(distribution=[hash[c]]) +- LocalHashWindowAggregate(groupBy=[c], window=[TumblingGroupWindow('w$, ts, 240000)], properties=[w$start, w$end, w$rowtime], select=[c]) +- Calc(select=[ts, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, ts]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/GroupingSetsTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/GroupingSetsTest.xml index f4c20840d8217..2b90453d5d495 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/GroupingSetsTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/GroupingSetsTest.xml @@ -28,7 +28,7 @@ FROM emp GROUP BY CUBE(MOD(deptno, 20), gender) LogicalProject(d=[$0], c=[$2], g=[$1]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], c=[COUNT()]) +- LogicalProject(d=[MOD($1, 20)], g=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -39,7 +39,7 @@ Calc(select=[d, c, g]) +- LocalHashAggregate(groupBy=[d, g, $e], select=[d, g, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{d, g, 0 AS $e}, {d, null AS g, 1 AS $e}, {null AS d, g, 2 AS $e}, {null AS d, null AS g, 3 AS $e}]) +- Calc(select=[MOD(deptno, 20) AS d, gender AS g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -52,7 +52,7 @@ Calc(select=[d, c, g]) LogicalProject(c=[$1]) +- LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()]) +- LogicalProject($f0=[1]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -63,7 +63,7 @@ Calc(select=[c]) +- LocalHashAggregate(groupBy=[$f0, $e], select=[$f0, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{$f0, 0 AS $e}, {null AS $f0, 1 AS $e}]) +- Calc(select=[1 AS $f0]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -79,7 +79,7 @@ FROM emp GROUP BY ROLLUP (deptno + 1) LogicalProject(d1=[$0], d0=[-($0, 1)], c=[$1]) +- LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()]) +- LogicalProject(d1=[+($1, 1)]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -90,7 +90,7 @@ Calc(select=[d1, (d1 - 1) AS d0, c]) +- LocalHashAggregate(groupBy=[d1, $e], select=[d1, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{d1, 0 AS $e}, {null AS d1, 1 AS $e}]) +- Calc(select=[(deptno + 1) AS d1]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -106,7 +106,7 @@ FROM emp GROUP BY ROLLUP(MOD(deptno, 20), gender) LogicalProject(d=[$0], c=[$2], g=[$1]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], c=[COUNT()]) +- LogicalProject(d=[MOD($1, 20)], g=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -117,7 +117,7 @@ Calc(select=[d, c, g]) +- LocalHashAggregate(groupBy=[d, g, $e], select=[d, g, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{d, g, 0 AS $e}, {d, null AS g, 1 AS $e}, {null AS d, null AS g, 3 AS $e}]) +- Calc(select=[MOD(deptno, 20) AS d, gender AS g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -130,7 +130,7 @@ Calc(select=[d, c, g]) LogicalProject(c=[$1]) +- LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()]) +- LogicalProject($f0=[1]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -141,7 +141,7 @@ Calc(select=[c]) +- LocalHashAggregate(groupBy=[$f0, $e], select=[$f0, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{$f0, 0 AS $e}, {null AS $f0, 1 AS $e}]) +- Calc(select=[1 AS $f0]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -153,7 +153,7 @@ Calc(select=[c]) @@ -164,7 +164,7 @@ Calc(select=[deptno, c]) +- LocalHashAggregate(groupBy=[deptno, $e], select=[deptno, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) +- Calc(select=[deptno]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emps, source: [TestTableSource(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)]]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat]) + +- TableSourceScan(table=[[default_catalog, default_database, emps]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat]) ]]> @@ -181,11 +181,11 @@ LogicalUnion(all=[true]) :- LogicalProject(deptno=[$0], g=[0:BIGINT], c=[$1]) : +- LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()]) : +- LogicalProject(deptno=[$7]) -: +- LogicalTableScan(table=[[default_catalog, default_database, scott_emp, source: [TestTableSource(empno, ename, job, mgr, hiredate, sal, comm, deptno)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, scott_emp]]) +- LogicalProject(deptno=[$0], g=[1:BIGINT], c=[$1]) +- LogicalAggregate(group=[{0}], groups=[[{}]], c=[COUNT()]) +- LogicalProject(deptno=[$7]) - +- LogicalTableScan(table=[[default_catalog, default_database, scott_emp, source: [TestTableSource(empno, ename, job, mgr, hiredate, sal, comm, deptno)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, scott_emp]]) ]]> @@ -197,7 +197,7 @@ Union(all=[true], union=[deptno, g, c]) : +- LocalHashAggregate(groupBy=[deptno, $e], select=[deptno, $e, Partial_COUNT(*) AS count1$0]) : +- Expand(projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) : +- Calc(select=[deptno])(reuse_id=[1]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, scott_emp, source: [TestTableSource(empno, ename, job, mgr, hiredate, sal, comm, deptno)]]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno]) +: +- TableSourceScan(table=[[default_catalog, default_database, scott_emp]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno]) +- Calc(select=[deptno, 1 AS g, c]) +- HashAggregate(isMerge=[true], groupBy=[deptno], select=[deptno, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[deptno]]) @@ -226,7 +226,7 @@ FROM MyTable LogicalProject(b=[$0], c=[$1], a=[$2], g=[0:BIGINT], gb=[$3], gc=[$4], gib=[$5], gic=[$6], gid=[$7]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], a=[AVG($2)], gb=[GROUPING($0)], gc=[GROUPING($1)], gib=[GROUPING_ID($0)], gic=[GROUPING_ID($1)], gid=[GROUPING_ID($0, $1)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -236,7 +236,7 @@ Calc(select=[b, c, a, 0 AS g, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS gb, CASE(($e +- Exchange(distribution=[hash[b, c, $e]]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)]) +- Expand(projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -250,7 +250,7 @@ LogicalAggregate(group=[{0}]) +- LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], EXPR$0=[COUNT()]) +- LogicalProject(deptno=[$1], gender=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -264,7 +264,7 @@ HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) +- LocalHashAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -285,8 +285,8 @@ LogicalProject(deptno=[$0], gender=[$2], min_name=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0, 2}, {0}, {1, 2}, {1}, {2}, {}]], min_name=[MIN($3)], agg#1=[COUNT()]) +- LogicalProject(deptno=[$1], deptno0=[$3], gender=[$2], ename=[$0]) +- LogicalJoin(condition=[=($1, $3)], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, dept, source: [TestTableSource(deptno, dname)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, emp]]) + +- LogicalTableScan(table=[[default_catalog, default_database, dept]]) ]]> @@ -302,10 +302,10 @@ Calc(select=[deptno, gender, min_name], where=[(($f5 > 2) OR ((gender = 'M') AND +- Expand(projects=[{ename, deptno, gender, deptno0, 0 AS $e}, {ename, deptno, gender, null AS deptno0, 1 AS $e}, {ename, deptno, null AS gender, deptno0, 2 AS $e}, {ename, deptno, null AS gender, null AS deptno0, 3 AS $e}, {ename, null AS deptno, gender, deptno0, 4 AS $e}, {ename, null AS deptno, gender, null AS deptno0, 5 AS $e}, {ename, null AS deptno, null AS gender, deptno0, 6 AS $e}, {ename, null AS deptno, null AS gender, null AS deptno0, 7 AS $e}]) +- HashJoin(joinType=[InnerJoin], where=[(deptno = deptno0)], select=[ename, deptno, gender, deptno0], build=[right]) :- Exchange(distribution=[hash[deptno]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + : +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) +- Exchange(distribution=[hash[deptno]]) +- Calc(select=[deptno]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, dept, source: [TestTableSource(deptno, dname)]]], fields=[deptno, dname]) + +- TableSourceScan(table=[[default_catalog, default_database, dept]], fields=[deptno, dname]) ]]> @@ -329,7 +329,7 @@ FROM MyTable LogicalProject(b=[$0], c=[$1], a=[$2], g=[0:BIGINT], gb=[$3], gc=[$4], gib=[$5], gic=[$6], gid=[$7]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], a=[AVG($2)], gb=[GROUPING($0)], gc=[GROUPING($1)], gib=[GROUPING_ID($0)], gic=[GROUPING_ID($1)], gid=[GROUPING_ID($0, $1)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -339,7 +339,7 @@ Calc(select=[b, c, a, 0 AS g, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS gb, CASE(($e +- Exchange(distribution=[hash[b, c, $e]]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)]) +- Expand(projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -359,7 +359,7 @@ LogicalProject(deptno=[$1], gender=[$0], EXPR$2=[$2], c=[$3]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[GROUPING_ID($1, $0, $1)], c=[COUNT()]) +- LogicalProject(gender=[$2], deptno=[$1]) +- LogicalFilter(condition=[=($1, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -370,7 +370,7 @@ Calc(select=[deptno, gender, CASE(($e = 0), 0, ($e = 1), 5, 7) AS EXPR$2, c]) +- LocalHashAggregate(groupBy=[gender, deptno, $e], select=[gender, deptno, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{gender, deptno, 0 AS $e}, {gender, null AS deptno, 1 AS $e}, {null AS gender, null AS deptno, 3 AS $e}]) +- Calc(select=[gender, CAST(10 AS INTEGER) AS deptno], where=[(deptno = 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -383,7 +383,7 @@ Calc(select=[deptno, gender, CASE(($e = 0), 0, ($e = 1), 5, 7) AS EXPR$2, c]) LogicalProject(EXPR$0=[+($0, 1)], c=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], c=[COUNT()]) +- LogicalProject(deptno=[$1], gender=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -394,7 +394,7 @@ Calc(select=[(deptno + 1) AS EXPR$0, c]) +- LocalHashAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -419,7 +419,7 @@ LogicalSort(sort0=[$0], sort1=[$1], sort2=[$2], dir0=[ASC-nulls-first], dir1=[AS +- LogicalProject(deptno=[$0], job=[$1], empno=[$2], ename=[$3], sumsal=[$4], gr_text=[CASE(=($5, 0), _UTF-16LE'grouped by deptno,job,empno,ename':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", =($5, 1), _UTF-16LE'grouped by deptno,job':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", =($5, 3), _UTF-16LE'grouped by deptno':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", =($5, 7), _UTF-16LE'grouped by ()':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", null:VARCHAR(2147483647) CHARACTER SET "UTF-16LE")]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 1}, {0}, {}]], sumsal=[SUM($4)], agg#1=[GROUPING_ID($0, $1, $2)]) +- LogicalProject(deptno=[$7], job=[$2], empno=[$0], ename=[$1], sal=[$5]) - +- LogicalTableScan(table=[[default_catalog, default_database, scott_emp, source: [TestTableSource(empno, ename, job, mgr, hiredate, sal, comm, deptno)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, scott_emp]]) ]]> @@ -432,7 +432,7 @@ Sort(orderBy=[deptno ASC, job ASC, empno ASC]) +- LocalHashAggregate(groupBy=[deptno, job, empno, ename, $e], select=[deptno, job, empno, ename, $e, Partial_SUM(sal) AS sum$0]) +- Expand(projects=[{deptno, job, empno, ename, sal, 0 AS $e}, {deptno, job, null AS empno, null AS ename, sal, 3 AS $e}, {deptno, null AS job, null AS empno, null AS ename, sal, 7 AS $e}, {null AS deptno, null AS job, null AS empno, null AS ename, sal, 15 AS $e}]) +- Calc(select=[deptno, job, empno, ename, sal]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, scott_emp, source: [TestTableSource(empno, ename, job, mgr, hiredate, sal, comm, deptno)]]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno]) + +- TableSourceScan(table=[[default_catalog, default_database, scott_emp]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno]) ]]> @@ -455,7 +455,7 @@ LogicalProject(c=[$2], gd=[$3], gid=[$4], gdgd=[$5], gidgd=[$6]) +- LogicalFilter(condition=[<=($3, $6)]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], c=[COUNT()], gd=[GROUPING($0)], gid=[GROUPING_ID($0)], gdgd=[GROUPING($0, $1, $0)], gidgd=[GROUPING_ID($0, $1, $0)]) +- LogicalProject(deptno=[$1], gender=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -467,7 +467,7 @@ Calc(select=[c, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS gd, CASE(SEARCH($e, Sarg[0 +- Calc(select=[deptno, gender, $e], where=[(CASE(SEARCH($e, Sarg[0, 1]), 0, 1) <= CASE(($e = 0), 0, ($e = 1), 2, 7))]) +- Expand(projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -490,7 +490,7 @@ LogicalProject(c=[$2], g=[$3], gid=[$4], gd=[$5], gg=[$6], ggd=[$7], gdg=[$8]) +- LogicalProject(deptno=[$0], gender=[$1], c=[$2], g=[$3], gid=[0:BIGINT], gd=[$4], gg=[$5], ggd=[$6], gdg=[$7]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], c=[COUNT()], g=[GROUPING($0)], gd=[GROUPING_ID($0)], gg=[GROUPING_ID($1)], ggd=[GROUPING_ID($1, $0)], gdg=[GROUPING_ID($0, $1)]) +- LogicalProject(deptno=[$1], gender=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -501,7 +501,7 @@ Calc(select=[c, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS g, 0 AS gid, CASE(SEARCH($ +- LocalHashAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -525,7 +525,7 @@ LogicalProject(deptno=[$0], gender=[$1], gd=[$2], gg=[$3], dg=[$4], gd0=[$5], gi +- LogicalProject(deptno=[$0], gender=[$1], gd=[$2], gg=[$3], dg=[$4], gd_0=[$5], gid=[0:BIGINT], c=[$6]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], gd=[GROUPING($0)], gg=[GROUPING($1)], dg=[GROUPING_ID($0, $1)], gd=[GROUPING_ID($1, $0)], c=[COUNT()]) +- LogicalProject(deptno=[$1], gender=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -536,7 +536,7 @@ Calc(select=[deptno, gender, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS gd, CASE(($e +- LocalHashAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -556,7 +556,7 @@ FROM scott_emp GROUP BY CUBE(deptno, job) @@ -567,7 +567,7 @@ Calc(select=[deptno, job, c, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS d, CASE(($e = +- LocalHashAggregate(groupBy=[deptno, job, $e], select=[deptno, job, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{deptno, job, 0 AS $e}, {deptno, null AS job, 1 AS $e}, {null AS deptno, job, 2 AS $e}, {null AS deptno, null AS job, 3 AS $e}]) +- Calc(select=[deptno, job]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, scott_emp, source: [TestTableSource(empno, ename, job, mgr, hiredate, sal, comm, deptno)]]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno]) + +- TableSourceScan(table=[[default_catalog, default_database, scott_emp]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno]) ]]> @@ -580,7 +580,7 @@ Calc(select=[deptno, job, c, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS d, CASE(($e = LogicalProject(c=[$1], g=[$2]) +- LogicalAggregate(group=[{0}], c=[COUNT()], g=[GROUPING($0)]) +- LogicalProject(deptno=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -590,7 +590,7 @@ Calc(select=[c, 0 AS g]) +- Exchange(distribution=[hash[deptno]]) +- LocalHashAggregate(groupBy=[deptno], select=[deptno, Partial_COUNT(*) AS count1$0]) +- Calc(select=[deptno]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -605,7 +605,7 @@ SELECT COUNT(*) AS c, deptno, GROUPING(deptno) AS g FROM emp GROUP BY ROLLUP(dep LogicalProject(c=[$1], deptno=[$0], g=[$2]) +- LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()], g=[GROUPING($0)]) +- LogicalProject(deptno=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -616,7 +616,7 @@ Calc(select=[c, deptno, CASE(($e = 0), 0, 1) AS g]) +- LocalHashAggregate(groupBy=[deptno, $e], select=[deptno, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) +- Calc(select=[deptno]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -632,7 +632,7 @@ GROUP BY GROUPING SETS (b, c) LogicalProject(b=[$0], c=[$1], a=[$2], g=[0:BIGINT]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}]], a=[AVG($2)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -642,7 +642,7 @@ Calc(select=[b, c, a, 0 AS g]) +- Exchange(distribution=[hash[b, c, $e]]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)]) +- Expand(projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -654,7 +654,7 @@ Calc(select=[b, c, a, 0 AS g]) @@ -664,7 +664,7 @@ Calc(select=[b, c, a]) +- Exchange(distribution=[hash[b, c, $e]]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)]) +- Expand(projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -689,7 +689,7 @@ FROM MyTable LogicalProject(b=[$0], c=[$1], a=[$2], g=[0:BIGINT], gb=[$3], gc=[$4], gib=[$5], gic=[$6], gid=[$7], cnt=[$8]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}, {}]], a=[AVG($2)], gb=[GROUPING($0)], gc=[GROUPING($1)], gib=[GROUPING_ID($0)], gic=[GROUPING_ID($1)], gid=[GROUPING_ID($0, $1)], cnt=[COUNT()]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -699,7 +699,7 @@ Calc(select=[b, c, a, 0 AS g, CASE(($e = 1), 0, 1) AS gb, CASE(($e = 1), 1, ($e +- Exchange(distribution=[hash[b, c, $e]]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1), Partial_COUNT(*) AS count1$2]) +- Expand(projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -713,7 +713,7 @@ SELECT deptno + 1, COUNT(*) AS c FROM emps GROUP BY GROUPING SETS ((), (deptno + @@ -724,7 +724,7 @@ Calc(select=[EXPR$0, c]) +- LocalHashAggregate(groupBy=[EXPR$0, $e], select=[EXPR$0, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{EXPR$0, 0 AS $e}, {null AS EXPR$0, 1 AS $e}]) +- Calc(select=[(deptno + 1) AS EXPR$0]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emps, source: [TestTableSource(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)]]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat]) + +- TableSourceScan(table=[[default_catalog, default_database, emps]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat]) ]]> @@ -741,7 +741,7 @@ LogicalProject(c=[$0]) +- LogicalProject(c=[$1], EXPR$1=[$2]) +- LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()], agg#1=[GROUPING($0)]) +- LogicalProject(deptno=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -755,7 +755,7 @@ Calc(select=[c]) +- LocalHashAggregate(groupBy=[deptno, $e], select=[deptno, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) +- Calc(select=[deptno]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -805,7 +805,7 @@ FROM emps GROUP BY GROUPING SETS (deptno) LogicalProject(deptno=[$0], a=[$1], g=[0:BIGINT], gb=[$2], gib=[$3]) +- LogicalAggregate(group=[{0}], a=[AVG($1)], gb=[GROUPING($0)], gib=[GROUPING_ID($0)]) +- LogicalProject(deptno=[$2], age=[$6]) - +- LogicalTableScan(table=[[default_catalog, default_database, emps, source: [TestTableSource(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emps]]) ]]> @@ -815,7 +815,7 @@ Calc(select=[deptno, a, 0 AS g, 0 AS gb, 0 AS gib]) +- Exchange(distribution=[hash[deptno]]) +- LocalHashAggregate(groupBy=[deptno], select=[deptno, Partial_AVG(age) AS (sum$0, count$1)]) +- Calc(select=[deptno, age]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emps, source: [TestTableSource(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)]]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat]) + +- TableSourceScan(table=[[default_catalog, default_database, emps]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat]) ]]> @@ -839,7 +839,7 @@ FROM MyTable LogicalProject(b=[$0], c=[$1], a=[$2], g=[0:BIGINT], gb=[$3], gc=[$4], gib=[$5], gic=[$6], gid=[$7]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], a=[AVG($2)], gb=[GROUPING($0)], gc=[GROUPING($1)], gib=[GROUPING_ID($0)], gic=[GROUPING_ID($1)], gid=[GROUPING_ID($0, $1)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -849,7 +849,7 @@ Calc(select=[b, c, a, 0 AS g, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS gb, CASE(($e +- Exchange(distribution=[hash[b, c, $e]]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)]) +- Expand(projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -873,7 +873,7 @@ FROM MyTable LogicalProject(b=[$0], c=[$1], a=[$2], g=[0:BIGINT], gb=[$3], gc=[$4], gib=[$5], gic=[$6], gid=[$7]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], a=[AVG($2)], gb=[GROUPING($0)], gc=[GROUPING($1)], gib=[GROUPING_ID($0)], gic=[GROUPING_ID($1)], gid=[GROUPING_ID($0, $1)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -883,7 +883,7 @@ Calc(select=[b, c, a, 0 AS g, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS gb, CASE(($e +- Exchange(distribution=[hash[b, c, $e]]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)]) +- Expand(projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -896,7 +896,7 @@ Calc(select=[b, c, a, 0 AS g, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS gb, CASE(($e LogicalProject(deptno=[$0], c=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], c=[COUNT()]) +- LogicalProject(deptno=[$1], gender=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -907,7 +907,7 @@ Calc(select=[deptno, c]) +- LocalHashAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -923,7 +923,7 @@ GROUP BY ROLLUP(deptno / 2, gender), ROLLUP(substring(ename FROM 1 FOR 1)) LogicalProject(half1=[+($0, 1)], c=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0, 2}, {0}, {2}, {}]], c=[COUNT()]) +- LogicalProject($f0=[/($1, 2)], gender=[$2], $f2=[SUBSTRING($0, 1, 1)]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -934,7 +934,7 @@ Calc(select=[($f0 + 1) AS half1, c]) +- LocalHashAggregate(groupBy=[$f0, gender, $f2, $e], select=[$f0, gender, $f2, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{$f0, gender, $f2, 0 AS $e}, {$f0, gender, null AS $f2, 1 AS $e}, {$f0, null AS gender, $f2, 2 AS $e}, {$f0, null AS gender, null AS $f2, 3 AS $e}, {null AS $f0, null AS gender, $f2, 6 AS $e}, {null AS $f0, null AS gender, null AS $f2, 7 AS $e}]) +- Calc(select=[(deptno / 2) AS $f0, gender, SUBSTRING(ename, 1, 1) AS $f2]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -949,7 +949,7 @@ SELECT gender, deptno + 1, COUNT(*) AS c FROM emp GROUP BY ROLLUP(deptno, gender LogicalProject(gender=[$1], EXPR$1=[+($0, 1)], c=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], c=[COUNT()]) +- LogicalProject(deptno=[$1], gender=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -960,7 +960,7 @@ Calc(select=[gender, (deptno + 1) AS EXPR$1, c]) +- LocalHashAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -972,7 +972,7 @@ Calc(select=[gender, (deptno + 1) AS EXPR$1, c]) @@ -983,7 +983,7 @@ Calc(select=[gender, c]) +- LocalHashAggregate(groupBy=[gender, $e], select=[gender, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{gender, 0 AS $e}, {null AS gender, 1 AS $e}]) +- Calc(select=[gender]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -996,7 +996,7 @@ Calc(select=[gender, c]) LogicalSort(sort0=[$1], dir0=[DESC-nulls-last]) +- LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()]) +- LogicalProject(gender=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -1009,7 +1009,7 @@ Sort(orderBy=[c DESC]) +- LocalHashAggregate(groupBy=[gender, $e], select=[gender, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{gender, 0 AS $e}, {null AS gender, 1 AS $e}]) +- Calc(select=[gender]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -1025,7 +1025,7 @@ LogicalProject(d1=[+($0, 1)], c=[$1]) +- LogicalFilter(condition=[>($1, 3)]) +- LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()]) +- LogicalProject(deptno=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -1036,7 +1036,7 @@ Calc(select=[(deptno + 1) AS d1, c], where=[(c > 3)]) +- LocalHashAggregate(groupBy=[deptno, $e], select=[deptno, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) +- Calc(select=[deptno]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -1049,7 +1049,7 @@ Calc(select=[(deptno + 1) AS d1, c], where=[(c > 3)]) LogicalProject(EXPR$0=[+($0, 1)], c=[$1]) +- LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()]) +- LogicalProject(deptno=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -1060,7 +1060,7 @@ Calc(select=[(deptno + 1) AS EXPR$0, c]) +- LocalHashAggregate(groupBy=[deptno, $e], select=[deptno, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) +- Calc(select=[deptno]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.xml index d05228dfaaa04..be082cc37ebc8 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.xml @@ -24,7 +24,7 @@ limitations under the License. @@ -32,7 +32,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[var_sum($1)]) HashAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[b, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -44,7 +44,7 @@ HashAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1]) @@ -52,7 +52,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[var_sum($1)]) HashAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[b, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -64,7 +64,7 @@ HashAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1]) @@ -72,7 +72,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[var_sum($1)]) HashAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[b, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -84,7 +84,7 @@ HashAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1]) @@ -93,7 +93,7 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3]) +- Calc(select=[a, b, c], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -105,7 +105,7 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final @@ -113,7 +113,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($ HashAggregate(isMerge=[false], select=[AVG(a) AS EXPR$0, SUM(b) AS EXPR$1, COUNT(c) AS EXPR$2]) +- Exchange(distribution=[single]) +- Calc(select=[a, b, c], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -125,7 +125,7 @@ HashAggregate(isMerge=[false], select=[AVG(a) AS EXPR$0, SUM(b) AS EXPR$1, COUNT @@ -134,7 +134,7 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3]) +- Calc(select=[a, b, c], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -147,7 +147,7 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($2)], EXPR$3=[SUM($3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[$2._1]) +- LogicalFilter(condition=[=($0, 1)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -156,7 +156,7 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3, Partial_SUM($f3) AS sum$4]) +- Calc(select=[CAST(1 AS INTEGER) AS a, b, c, c._1 AS $f3], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c]) ]]> @@ -169,7 +169,7 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($2)], EXPR$3=[SUM($3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[$2._1]) +- LogicalFilter(condition=[=($0, 1)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -177,7 +177,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($ HashAggregate(isMerge=[false], select=[AVG(a) AS EXPR$0, SUM(b) AS EXPR$1, COUNT(c) AS EXPR$2, SUM($f3) AS EXPR$3]) +- Exchange(distribution=[single]) +- Calc(select=[CAST(1 AS INTEGER) AS a, b, c, c._1 AS $f3], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c]) ]]> @@ -190,7 +190,7 @@ HashAggregate(isMerge=[false], select=[AVG(a) AS EXPR$0, SUM(b) AS EXPR$1, COUNT LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($2)], EXPR$3=[SUM($3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[$2._1]) +- LogicalFilter(condition=[=($0, 1)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -199,7 +199,7 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3, Partial_SUM($f3) AS sum$4]) +- Calc(select=[CAST(1 AS INTEGER) AS a, b, c, c._1 AS $f3], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c]) ]]> @@ -211,7 +211,7 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final @@ -220,7 +220,7 @@ HashAggregate(isMerge=[true], groupBy=[a, b], select=[a, b]) +- Exchange(distribution=[hash[a, b]]) +- LocalHashAggregate(groupBy=[a, b], select=[a, b]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -232,7 +232,7 @@ HashAggregate(isMerge=[true], groupBy=[a, b], select=[a, b]) @@ -240,7 +240,7 @@ LogicalAggregate(group=[{0, 1}]) HashAggregate(isMerge=[false], groupBy=[a, b], select=[a, b]) +- Exchange(distribution=[hash[a, b]]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -252,7 +252,7 @@ HashAggregate(isMerge=[false], groupBy=[a, b], select=[a, b]) @@ -261,7 +261,7 @@ HashAggregate(isMerge=[true], groupBy=[a, b], select=[a, b]) +- Exchange(distribution=[hash[a, b]]) +- LocalHashAggregate(groupBy=[a, b], select=[a, b]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -272,7 +272,7 @@ HashAggregate(isMerge=[true], groupBy=[a, b], select=[a, b]) @@ -280,7 +280,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final_SUM(sum$2) AS EXPR$1, Final_COUNT(count$3) AS EXPR$2]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -291,14 +291,14 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final @@ -309,7 +309,7 @@ HashAggregate(isMerge=[false], select=[AVG(a) AS EXPR$0, SUM(b) AS EXPR$1, COUNT @@ -317,7 +317,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final_SUM(sum$2) AS EXPR$1, Final_COUNT(count$3) AS EXPR$2]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -339,7 +339,7 @@ FROM MyTable @@ -348,7 +348,7 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT sum$0, BIGINT count$1, BIGINT sum$2, BIGINT count$3, BIGINT sum$4, BIGINT count$5, BIGINT sum$6, BIGINT count$7, DOUBLE sum$8, BIGINT count$9, DOUBLE sum$10, BIGINT count$11, DECIMAL(38, 20) sum$12, BIGINT count$13, DECIMAL(38, 5) sum$14, BIGINT count$15)] +- LocalHashAggregate(select=[Partial_AVG(byte) AS (sum$0, count$1), Partial_AVG(short) AS (sum$2, count$3), Partial_AVG(int) AS (sum$4, count$5), Partial_AVG(long) AS (sum$6, count$7), Partial_AVG(float) AS (sum$8, count$9), Partial_AVG(double) AS (sum$10, count$11), Partial_AVG(decimal3020) AS (sum$12, count$13), Partial_AVG(decimal105) AS (sum$14, count$15)]), rowType=[RecordType(BIGINT sum$0, BIGINT count$1, BIGINT sum$2, BIGINT count$3, BIGINT sum$4, BIGINT count$5, BIGINT sum$6, BIGINT count$7, DOUBLE sum$8, BIGINT count$9, DOUBLE sum$10, BIGINT count$11, DECIMAL(38, 20) sum$12, BIGINT count$13, DECIMAL(38, 5) sum$14, BIGINT count$15)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -370,7 +370,7 @@ FROM MyTable @@ -378,7 +378,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[AVG($1)], EXPR$2=[AVG($2) HashAggregate(isMerge=[false], select=[AVG(byte) AS EXPR$0, AVG(short) AS EXPR$1, AVG(int) AS EXPR$2, AVG(long) AS EXPR$3, AVG(float) AS EXPR$4, AVG(double) AS EXPR$5, AVG(decimal3020) AS EXPR$6, AVG(decimal105) AS EXPR$7]), rowType=[RecordType(TINYINT EXPR$0, SMALLINT EXPR$1, INTEGER EXPR$2, BIGINT EXPR$3, FLOAT EXPR$4, DOUBLE EXPR$5, DECIMAL(38, 20) EXPR$6, DECIMAL(38, 6) EXPR$7)] +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -400,7 +400,7 @@ FROM MyTable @@ -409,7 +409,7 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT sum$0, BIGINT count$1, BIGINT sum$2, BIGINT count$3, BIGINT sum$4, BIGINT count$5, BIGINT sum$6, BIGINT count$7, DOUBLE sum$8, BIGINT count$9, DOUBLE sum$10, BIGINT count$11, DECIMAL(38, 20) sum$12, BIGINT count$13, DECIMAL(38, 5) sum$14, BIGINT count$15)] +- LocalHashAggregate(select=[Partial_AVG(byte) AS (sum$0, count$1), Partial_AVG(short) AS (sum$2, count$3), Partial_AVG(int) AS (sum$4, count$5), Partial_AVG(long) AS (sum$6, count$7), Partial_AVG(float) AS (sum$8, count$9), Partial_AVG(double) AS (sum$10, count$11), Partial_AVG(decimal3020) AS (sum$12, count$13), Partial_AVG(decimal105) AS (sum$14, count$15)]), rowType=[RecordType(BIGINT sum$0, BIGINT count$1, BIGINT sum$2, BIGINT count$3, BIGINT sum$4, BIGINT count$5, BIGINT sum$6, BIGINT count$7, DOUBLE sum$8, BIGINT count$9, DOUBLE sum$10, BIGINT count$11, DECIMAL(38, 20) sum$12, BIGINT count$13, DECIMAL(38, 5) sum$14, BIGINT count$15)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -436,7 +436,7 @@ FROM MyTable @@ -444,7 +444,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT($0)], EXPR$1=[COUNT($1)], EXPR$2=[COU HashAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_COUNT(count$1) AS EXPR$1, Final_COUNT(count$2) AS EXPR$2, Final_COUNT(count$3) AS EXPR$3, Final_COUNT(count$4) AS EXPR$4, Final_COUNT(count$5) AS EXPR$5, Final_COUNT(count$6) AS EXPR$6, Final_COUNT(count$7) AS EXPR$7, Final_COUNT(count$8) AS EXPR$8, Final_COUNT(count$9) AS EXPR$9, Final_COUNT(count$10) AS EXPR$10, Final_COUNT(count$11) AS EXPR$11, Final_COUNT(count$12) AS EXPR$12]), rowType=[RecordType(BIGINT EXPR$0, BIGINT EXPR$1, BIGINT EXPR$2, BIGINT EXPR$3, BIGINT EXPR$4, BIGINT EXPR$5, BIGINT EXPR$6, BIGINT EXPR$7, BIGINT EXPR$8, BIGINT EXPR$9, BIGINT EXPR$10, BIGINT EXPR$11, BIGINT EXPR$12)] +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count$0, BIGINT count$1, BIGINT count$2, BIGINT count$3, BIGINT count$4, BIGINT count$5, BIGINT count$6, BIGINT count$7, BIGINT count$8, BIGINT count$9, BIGINT count$10, BIGINT count$11, BIGINT count$12)] +- LocalHashAggregate(select=[Partial_COUNT(byte) AS count$0, Partial_COUNT(short) AS count$1, Partial_COUNT(int) AS count$2, Partial_COUNT(long) AS count$3, Partial_COUNT(float) AS count$4, Partial_COUNT(double) AS count$5, Partial_COUNT(decimal3020) AS count$6, Partial_COUNT(decimal105) AS count$7, Partial_COUNT(boolean) AS count$8, Partial_COUNT(date) AS count$9, Partial_COUNT(time) AS count$10, Partial_COUNT(timestamp) AS count$11, Partial_COUNT(string) AS count$12]), rowType=[RecordType(BIGINT count$0, BIGINT count$1, BIGINT count$2, BIGINT count$3, BIGINT count$4, BIGINT count$5, BIGINT count$6, BIGINT count$7, BIGINT count$8, BIGINT count$9, BIGINT count$10, BIGINT count$11, BIGINT count$12)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -471,14 +471,14 @@ FROM MyTable @@ -505,7 +505,7 @@ FROM MyTable @@ -513,7 +513,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT($0)], EXPR$1=[COUNT($1)], EXPR$2=[COU HashAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_COUNT(count$1) AS EXPR$1, Final_COUNT(count$2) AS EXPR$2, Final_COUNT(count$3) AS EXPR$3, Final_COUNT(count$4) AS EXPR$4, Final_COUNT(count$5) AS EXPR$5, Final_COUNT(count$6) AS EXPR$6, Final_COUNT(count$7) AS EXPR$7, Final_COUNT(count$8) AS EXPR$8, Final_COUNT(count$9) AS EXPR$9, Final_COUNT(count$10) AS EXPR$10, Final_COUNT(count$11) AS EXPR$11, Final_COUNT(count$12) AS EXPR$12]), rowType=[RecordType(BIGINT EXPR$0, BIGINT EXPR$1, BIGINT EXPR$2, BIGINT EXPR$3, BIGINT EXPR$4, BIGINT EXPR$5, BIGINT EXPR$6, BIGINT EXPR$7, BIGINT EXPR$8, BIGINT EXPR$9, BIGINT EXPR$10, BIGINT EXPR$11, BIGINT EXPR$12)] +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count$0, BIGINT count$1, BIGINT count$2, BIGINT count$3, BIGINT count$4, BIGINT count$5, BIGINT count$6, BIGINT count$7, BIGINT count$8, BIGINT count$9, BIGINT count$10, BIGINT count$11, BIGINT count$12)] +- LocalHashAggregate(select=[Partial_COUNT(byte) AS count$0, Partial_COUNT(short) AS count$1, Partial_COUNT(int) AS count$2, Partial_COUNT(long) AS count$3, Partial_COUNT(float) AS count$4, Partial_COUNT(double) AS count$5, Partial_COUNT(decimal3020) AS count$6, Partial_COUNT(decimal105) AS count$7, Partial_COUNT(boolean) AS count$8, Partial_COUNT(date) AS count$9, Partial_COUNT(time) AS count$10, Partial_COUNT(timestamp) AS count$11, Partial_COUNT(string) AS count$12]), rowType=[RecordType(BIGINT count$0, BIGINT count$1, BIGINT count$2, BIGINT count$3, BIGINT count$4, BIGINT count$5, BIGINT count$6, BIGINT count$7, BIGINT count$8, BIGINT count$9, BIGINT count$10, BIGINT count$11, BIGINT count$12)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -525,7 +525,7 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_COUN @@ -534,7 +534,7 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count1$0)] +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]), rowType=[RecordType(BIGINT count1$0)] +- Calc(select=[0 AS $f0]), rowType=[RecordType(INTEGER $f0)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -546,7 +546,7 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType @@ -554,7 +554,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]), rowType=[RecordType(BIGINT EXPR$ HashAggregate(isMerge=[false], select=[COUNT(*) AS EXPR$0]), rowType=[RecordType(BIGINT EXPR$0)] +- Exchange(distribution=[single]), rowType=[RecordType(INTEGER $f0)] +- Calc(select=[0 AS $f0]), rowType=[RecordType(INTEGER $f0)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -566,7 +566,7 @@ HashAggregate(isMerge=[false], select=[COUNT(*) AS EXPR$0]), rowType=[RecordType @@ -575,7 +575,7 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count1$0)] +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]), rowType=[RecordType(BIGINT count1$0)] +- Calc(select=[0 AS $f0]), rowType=[RecordType(INTEGER $f0)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -644,7 +644,7 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType @@ -652,7 +652,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT($2)]) HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1, Final_COUNT(count$1) AS EXPR$2]) +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_COUNT(c) AS count$1]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -663,14 +663,14 @@ HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1 @@ -681,7 +681,7 @@ HashAggregate(isMerge=[false], groupBy=[a], select=[a, SUM(b) AS EXPR$1, COUNT(c @@ -689,7 +689,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT($2)]) HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1, Final_COUNT(count$1) AS EXPR$2]) +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_COUNT(c) AS count$1]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -701,7 +701,7 @@ HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1 @@ -710,7 +710,7 @@ HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1 +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_COUNT(c) AS count$1]) +- Calc(select=[a, b, c], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -722,7 +722,7 @@ HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1 @@ -730,7 +730,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT($2)]) HashAggregate(isMerge=[false], groupBy=[a], select=[a, SUM(b) AS EXPR$1, COUNT(c) AS EXPR$2]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, c], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -742,7 +742,7 @@ HashAggregate(isMerge=[false], groupBy=[a], select=[a, SUM(b) AS EXPR$1, COUNT(c @@ -751,7 +751,7 @@ HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1 +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_COUNT(c) AS count$1]) +- Calc(select=[a, b, c], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -766,7 +766,7 @@ SELECT a, MAX(b), c FROM (SELECT a, 'test' AS c, b FROM MyTable1) t GROUP BY a, LogicalProject(a=[$0], EXPR$1=[$2], c=[$1]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[MAX($2)]) +- LogicalProject(a=[$0], c=[_UTF-16LE'test'], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -776,7 +776,7 @@ Calc(select=[a, EXPR$1, 'test' AS c]) +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a, Partial_MAX(b) AS max$0]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -791,7 +791,7 @@ SELECT a, MAX(b), c FROM (SELECT a, 'test' AS c, b FROM MyTable1) t GROUP BY a, LogicalProject(a=[$0], EXPR$1=[$2], c=[$1]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[MAX($2)]) +- LogicalProject(a=[$0], c=[_UTF-16LE'test'], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -800,7 +800,7 @@ Calc(select=[a, EXPR$1, 'test' AS c]) +- HashAggregate(isMerge=[false], groupBy=[a], select=[a, MAX(b) AS EXPR$1]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -815,7 +815,7 @@ SELECT a, MAX(b), c FROM (SELECT a, 'test' AS c, b FROM MyTable1) t GROUP BY a, LogicalProject(a=[$0], EXPR$1=[$2], c=[$1]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[MAX($2)]) +- LogicalProject(a=[$0], c=[_UTF-16LE'test'], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -825,7 +825,7 @@ Calc(select=[a, EXPR$1, 'test' AS c]) +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a, Partial_MAX(b) AS max$0]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -851,7 +851,7 @@ FROM MyTable @@ -860,7 +860,7 @@ HashAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0, Final_MAX(max$ +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT max$0, SMALLINT max$1, INTEGER max$2, BIGINT max$3, FLOAT max$4, DOUBLE max$5, DECIMAL(30, 20) max$6, DECIMAL(10, 5) max$7, BOOLEAN max$8, DATE max$9, TIME(0) max$10, TIMESTAMP(3) max$11)] +- LocalHashAggregate(select=[Partial_MAX(byte) AS max$0, Partial_MAX(short) AS max$1, Partial_MAX(int) AS max$2, Partial_MAX(long) AS max$3, Partial_MAX(float) AS max$4, Partial_MAX(double) AS max$5, Partial_MAX(decimal3020) AS max$6, Partial_MAX(decimal105) AS max$7, Partial_MAX(boolean) AS max$8, Partial_MAX(date) AS max$9, Partial_MAX(time) AS max$10, Partial_MAX(timestamp) AS max$11]), rowType=[RecordType(TINYINT max$0, SMALLINT max$1, INTEGER max$2, BIGINT max$3, FLOAT max$4, DOUBLE max$5, DECIMAL(30, 20) max$6, DECIMAL(10, 5) max$7, BOOLEAN max$8, DATE max$9, TIME(0) max$10, TIMESTAMP(3) max$11)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -886,7 +886,7 @@ FROM MyTable @@ -894,7 +894,7 @@ LogicalAggregate(group=[{}], EXPR$0=[MAX($0)], EXPR$1=[MAX($1)], EXPR$2=[MAX($2) HashAggregate(isMerge=[false], select=[MAX(byte) AS EXPR$0, MAX(short) AS EXPR$1, MAX(int) AS EXPR$2, MAX(long) AS EXPR$3, MAX(float) AS EXPR$4, MAX(double) AS EXPR$5, MAX(decimal3020) AS EXPR$6, MAX(decimal105) AS EXPR$7, MAX(boolean) AS EXPR$8, MAX(date) AS EXPR$9, MAX(time) AS EXPR$10, MAX(timestamp) AS EXPR$11]), rowType=[RecordType(TINYINT EXPR$0, SMALLINT EXPR$1, INTEGER EXPR$2, BIGINT EXPR$3, FLOAT EXPR$4, DOUBLE EXPR$5, DECIMAL(30, 20) EXPR$6, DECIMAL(10, 5) EXPR$7, BOOLEAN EXPR$8, DATE EXPR$9, TIME(0) EXPR$10, TIMESTAMP(3) EXPR$11)] +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -920,7 +920,7 @@ FROM MyTable @@ -929,7 +929,7 @@ HashAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0, Final_MAX(max$ +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT max$0, SMALLINT max$1, INTEGER max$2, BIGINT max$3, FLOAT max$4, DOUBLE max$5, DECIMAL(30, 20) max$6, DECIMAL(10, 5) max$7, BOOLEAN max$8, DATE max$9, TIME(0) max$10, TIMESTAMP(3) max$11)] +- LocalHashAggregate(select=[Partial_MAX(byte) AS max$0, Partial_MAX(short) AS max$1, Partial_MAX(int) AS max$2, Partial_MAX(long) AS max$3, Partial_MAX(float) AS max$4, Partial_MAX(double) AS max$5, Partial_MAX(decimal3020) AS max$6, Partial_MAX(decimal105) AS max$7, Partial_MAX(boolean) AS max$8, Partial_MAX(date) AS max$9, Partial_MAX(time) AS max$10, Partial_MAX(timestamp) AS max$11]), rowType=[RecordType(TINYINT max$0, SMALLINT max$1, INTEGER max$2, BIGINT max$3, FLOAT max$4, DOUBLE max$5, DECIMAL(30, 20) max$6, DECIMAL(10, 5) max$7, BOOLEAN max$8, DATE max$9, TIME(0) max$10, TIMESTAMP(3) max$11)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -955,7 +955,7 @@ FROM MyTable @@ -964,7 +964,7 @@ HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS EXPR$0, Final_MIN(min$ +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT min$0, SMALLINT min$1, INTEGER min$2, BIGINT min$3, FLOAT min$4, DOUBLE min$5, DECIMAL(30, 20) min$6, DECIMAL(10, 5) min$7, BOOLEAN min$8, DATE min$9, TIME(0) min$10, TIMESTAMP(3) min$11)] +- LocalHashAggregate(select=[Partial_MIN(byte) AS min$0, Partial_MIN(short) AS min$1, Partial_MIN(int) AS min$2, Partial_MIN(long) AS min$3, Partial_MIN(float) AS min$4, Partial_MIN(double) AS min$5, Partial_MIN(decimal3020) AS min$6, Partial_MIN(decimal105) AS min$7, Partial_MIN(boolean) AS min$8, Partial_MIN(date) AS min$9, Partial_MIN(time) AS min$10, Partial_MIN(timestamp) AS min$11]), rowType=[RecordType(TINYINT min$0, SMALLINT min$1, INTEGER min$2, BIGINT min$3, FLOAT min$4, DOUBLE min$5, DECIMAL(30, 20) min$6, DECIMAL(10, 5) min$7, BOOLEAN min$8, DATE min$9, TIME(0) min$10, TIMESTAMP(3) min$11)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -990,7 +990,7 @@ FROM MyTable @@ -998,7 +998,7 @@ LogicalAggregate(group=[{}], EXPR$0=[MIN($0)], EXPR$1=[MIN($1)], EXPR$2=[MIN($2) HashAggregate(isMerge=[false], select=[MIN(byte) AS EXPR$0, MIN(short) AS EXPR$1, MIN(int) AS EXPR$2, MIN(long) AS EXPR$3, MIN(float) AS EXPR$4, MIN(double) AS EXPR$5, MIN(decimal3020) AS EXPR$6, MIN(decimal105) AS EXPR$7, MIN(boolean) AS EXPR$8, MIN(date) AS EXPR$9, MIN(time) AS EXPR$10, MIN(timestamp) AS EXPR$11]), rowType=[RecordType(TINYINT EXPR$0, SMALLINT EXPR$1, INTEGER EXPR$2, BIGINT EXPR$3, FLOAT EXPR$4, DOUBLE EXPR$5, DECIMAL(30, 20) EXPR$6, DECIMAL(10, 5) EXPR$7, BOOLEAN EXPR$8, DATE EXPR$9, TIME(0) EXPR$10, TIMESTAMP(3) EXPR$11)] +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -1024,7 +1024,7 @@ FROM MyTable @@ -1033,7 +1033,7 @@ HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS EXPR$0, Final_MIN(min$ +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT min$0, SMALLINT min$1, INTEGER min$2, BIGINT min$3, FLOAT min$4, DOUBLE min$5, DECIMAL(30, 20) min$6, DECIMAL(10, 5) min$7, BOOLEAN min$8, DATE min$9, TIME(0) min$10, TIMESTAMP(3) min$11)] +- LocalHashAggregate(select=[Partial_MIN(byte) AS min$0, Partial_MIN(short) AS min$1, Partial_MIN(int) AS min$2, Partial_MIN(long) AS min$3, Partial_MIN(float) AS min$4, Partial_MIN(double) AS min$5, Partial_MIN(decimal3020) AS min$6, Partial_MIN(decimal105) AS min$7, Partial_MIN(boolean) AS min$8, Partial_MIN(date) AS min$9, Partial_MIN(time) AS min$10, Partial_MIN(timestamp) AS min$11]), rowType=[RecordType(TINYINT min$0, SMALLINT min$1, INTEGER min$2, BIGINT min$3, FLOAT min$4, DOUBLE min$5, DECIMAL(30, 20) min$6, DECIMAL(10, 5) min$7, BOOLEAN min$8, DATE min$9, TIME(0) min$10, TIMESTAMP(3) min$11)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -1055,7 +1055,7 @@ FROM MyTable @@ -1064,7 +1064,7 @@ HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS EXPR$0, Final_SUM(sum$ +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT sum$0, SMALLINT sum$1, INTEGER sum$2, BIGINT sum$3, FLOAT sum$4, DOUBLE sum$5, DECIMAL(38, 20) sum$6, DECIMAL(38, 5) sum$7)] +- LocalHashAggregate(select=[Partial_SUM(byte) AS sum$0, Partial_SUM(short) AS sum$1, Partial_SUM(int) AS sum$2, Partial_SUM(long) AS sum$3, Partial_SUM(float) AS sum$4, Partial_SUM(double) AS sum$5, Partial_SUM(decimal3020) AS sum$6, Partial_SUM(decimal105) AS sum$7]), rowType=[RecordType(TINYINT sum$0, SMALLINT sum$1, INTEGER sum$2, BIGINT sum$3, FLOAT sum$4, DOUBLE sum$5, DECIMAL(38, 20) sum$6, DECIMAL(38, 5) sum$7)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -1086,7 +1086,7 @@ FROM MyTable @@ -1094,7 +1094,7 @@ LogicalAggregate(group=[{}], EXPR$0=[SUM($0)], EXPR$1=[SUM($1)], EXPR$2=[SUM($2) HashAggregate(isMerge=[false], select=[SUM(byte) AS EXPR$0, SUM(short) AS EXPR$1, SUM(int) AS EXPR$2, SUM(long) AS EXPR$3, SUM(float) AS EXPR$4, SUM(double) AS EXPR$5, SUM(decimal3020) AS EXPR$6, SUM(decimal105) AS EXPR$7]), rowType=[RecordType(TINYINT EXPR$0, SMALLINT EXPR$1, INTEGER EXPR$2, BIGINT EXPR$3, FLOAT EXPR$4, DOUBLE EXPR$5, DECIMAL(38, 20) EXPR$6, DECIMAL(38, 5) EXPR$7)] +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -1116,7 +1116,7 @@ FROM MyTable @@ -1125,7 +1125,7 @@ HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS EXPR$0, Final_SUM(sum$ +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT sum$0, SMALLINT sum$1, INTEGER sum$2, BIGINT sum$3, FLOAT sum$4, DOUBLE sum$5, DECIMAL(38, 20) sum$6, DECIMAL(38, 5) sum$7)] +- LocalHashAggregate(select=[Partial_SUM(byte) AS sum$0, Partial_SUM(short) AS sum$1, Partial_SUM(int) AS sum$2, Partial_SUM(long) AS sum$3, Partial_SUM(float) AS sum$4, Partial_SUM(double) AS sum$5, Partial_SUM(decimal3020) AS sum$6, Partial_SUM(decimal105) AS sum$7]), rowType=[RecordType(TINYINT sum$0, SMALLINT sum$1, INTEGER sum$2, BIGINT sum$3, FLOAT sum$4, DOUBLE sum$5, DECIMAL(38, 20) sum$6, DECIMAL(38, 5) sum$7)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/OverAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/OverAggregateTest.xml index ed6b45f01fceb..d75657f64e530 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/OverAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/OverAggregateTest.xml @@ -31,7 +31,7 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST)], EXPR$2=[CAST(/(CASE(>(COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST), null:INTEGER), COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST))):INTEGER], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)], EXPR$4=[MIN($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST)]) -+- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -49,7 +49,7 @@ Calc(select=[CASE((w0$o0 > 0), w0$o1, null:INTEGER) AS EXPR$0, w1$o0 AS EXPR$1, +- Exchange(distribution=[forward]) +- Sort(orderBy=[b ASC, a ASC]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -68,7 +68,7 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST)], EXPR$2=[MIN($0) OVER (ORDER BY $2 NULLS FIRST, $0 NULLS FIRST)], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST)], EXPR$4=[CAST(/(CASE(>(COUNT($0) OVER (ORDER BY $1 NULLS FIRST), 0), $SUM0($0) OVER (ORDER BY $1 NULLS FIRST), null:INTEGER), COUNT($0) OVER (ORDER BY $1 NULLS FIRST))):INTEGER]) -+- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -89,7 +89,7 @@ Calc(select=[CASE((w0$o0 > 0), w0$o1, null:INTEGER) AS EXPR$0, w1$o0 AS EXPR$1, +- OverAggregate(orderBy=[b ASC], window#0=[COUNT(a) AS w0$o2, $SUM0(a) AS w1$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o2, w1$o0]) +- Sort(orderBy=[b ASC]) +- Exchange(distribution=[single]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -105,7 +105,7 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST)]) -+- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -119,7 +119,7 @@ Calc(select=[CASE((w0$o0 > 0), w0$o1, null:INTEGER) AS EXPR$0, w1$o0 AS EXPR$1]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[b ASC, a ASC]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -138,7 +138,7 @@ SELECT (COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST), null:INTEGER)], EXPR$2=[RANK() OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST, $2 NULLS FIRST)], EXPR$3=[CASE(>(COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING), 0), $SUM0($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING), null:INTEGER)], EXPR$4=[COUNT() OVER (PARTITION BY $2 ORDER BY $2 NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING)]) -+- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -151,7 +151,7 @@ Calc(select=[w0$o0 AS EXPR$0, CASE((w1$o0 > 0), w1$o1, null:INTEGER) AS EXPR$1, +- Sort(orderBy=[c ASC, a ASC]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -166,7 +166,7 @@ FROM MyTable @@ -177,7 +177,7 @@ Calc(select=[c, w0$o0 AS $1]) +- Sort(orderBy=[c ASC, a ASC]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -192,7 +192,7 @@ FROM MyTable @@ -203,7 +203,7 @@ Calc(select=[c, w0$o0 AS $1]) +- Sort(orderBy=[c ASC, a ASC]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -218,7 +218,7 @@ FROM MyTable @@ -229,7 +229,7 @@ Calc(select=[c, w0$o0 AS $1]) +- Sort(orderBy=[c ASC, a ASC]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -244,7 +244,7 @@ FROM MyTable @@ -255,7 +255,7 @@ Calc(select=[w0$o0 AS $0]) +- Sort(orderBy=[c ASC, a ASC]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -266,7 +266,7 @@ Calc(select=[w0$o0 AS $0]) @@ -276,7 +276,7 @@ OverAggregate(partitionBy=[c], window#0=[COUNT(*) AS w0$o0 RANG BETWEEN UNBOUNDE +- Sort(orderBy=[c ASC]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -287,7 +287,7 @@ OverAggregate(partitionBy=[c], window#0=[COUNT(*) AS w0$o0 RANG BETWEEN UNBOUNDE @@ -298,7 +298,7 @@ Calc(select=[a, w0$o0 AS $1]) +- Sort(orderBy=[a ASC, proctime ASC]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, proctime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTableWithProctime, source: [TestTableSource(a, b, c, proctime)]]], fields=[a, b, c, proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTableWithProctime]], fields=[a, b, c, proctime]) ]]> @@ -309,7 +309,7 @@ Calc(select=[a, w0$o0 AS $1]) @@ -320,7 +320,7 @@ Calc(select=[a, w0$o0 AS $1]) +- Sort(orderBy=[a ASC, proctime ASC]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, proctime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTableWithProctime, source: [TestTableSource(a, b, c, proctime)]]], fields=[a, b, c, proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTableWithProctime]], fields=[a, b, c, proctime]) ]]> @@ -331,7 +331,7 @@ Calc(select=[a, w0$o0 AS $1]) (COUNT($0) OVER (ORDER BY $1 NULLS FIRST), 0), $SUM0($0) OVER (ORDER BY $1 NULLS FIRST), null:INTEGER)]) -+- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -340,7 +340,7 @@ Calc(select=[c, CASE((w0$o0 > 0), w0$o1, null:INTEGER) AS EXPR$1]) +- OverAggregate(orderBy=[b ASC], window#0=[COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0, w0$o1]) +- Sort(orderBy=[b ASC]) +- Exchange(distribution=[single]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -351,7 +351,7 @@ Calc(select=[c, CASE((w0$o0 > 0), w0$o1, null:INTEGER) AS EXPR$1]) @@ -359,7 +359,7 @@ LogicalProject(c=[$2], EXPR$1=[COUNT() OVER ()]) OverAggregate(window#0=[COUNT(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[c, w0$o0]) +- Exchange(distribution=[single]) +- Calc(select=[c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -374,7 +374,7 @@ FROM MyTable @@ -385,7 +385,7 @@ Calc(select=[w0$o0 AS $0]) +- Sort(orderBy=[c ASC, a ASC]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -400,7 +400,7 @@ FROM MyTable @@ -411,7 +411,7 @@ Calc(select=[w0$o0 AS $0]) +- Sort(orderBy=[c ASC, a ASC]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -422,7 +422,7 @@ Calc(select=[w0$o0 AS $0]) @@ -433,7 +433,7 @@ Calc(select=[w0$o0 AS $0]) +- Sort(orderBy=[c ASC, a ASC]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -444,7 +444,7 @@ Calc(select=[w0$o0 AS $0]) @@ -452,7 +452,7 @@ LogicalProject(EXPR$0=[COUNT(1) OVER ()]) OverAggregate(window#0=[COUNT(1) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[w0$o0]) +- Exchange(distribution=[single]) +- Calc(select=[]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -468,7 +468,7 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN -1 FOLLOWING AND 10 FOLLOWING), 0), $SUM0($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN -1 FOLLOWING AND 10 FOLLOWING), null:INTEGER)], EXPR$1=[COUNT(1) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN -1 FOLLOWING AND 10 FOLLOWING)]) -+- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -479,7 +479,7 @@ Calc(select=[CASE((w0$o0 > 0), w0$o1, null:INTEGER) AS EXPR$0, w0$o2 AS EXPR$1]) +- Sort(orderBy=[c ASC, a ASC]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -495,7 +495,7 @@ FROM MyTable @@ -509,7 +509,7 @@ Calc(select=[w0$o0 AS $0, w1$o0 AS $1]) +- Sort(orderBy=[a ASC]) +- Exchange(distribution=[single]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -528,7 +528,7 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $1 NULLS FIRST)], EXPR$2=[CAST(/(CASE(>(COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), null:INTEGER), COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST))):INTEGER], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST)], EXPR$4=[MIN($0) OVER (PARTITION BY $1 ORDER BY $1 NULLS FIRST)]) -+- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -544,7 +544,7 @@ Calc(select=[CASE((w0$o0 > 0), w0$o1, null:INTEGER) AS EXPR$0, w1$o0 AS EXPR$1, +- Exchange(distribution=[forward]) +- Sort(orderBy=[b ASC, a ASC]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -563,7 +563,7 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)], EXPR$2=[CAST(/(CASE(>(COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST, $2 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST, $2 NULLS FIRST), null:INTEGER), COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST, $2 NULLS FIRST))):INTEGER], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST, $1 NULLS FIRST)], EXPR$4=[MIN($0) OVER (PARTITION BY $1 ORDER BY $1 NULLS FIRST)]) -+- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -583,7 +583,7 @@ Calc(select=[CASE((w0$o0 > 0), w0$o1, null:INTEGER) AS EXPR$0, w1$o0 AS EXPR$1, +- Exchange(distribution=[forward]) +- Sort(orderBy=[b ASC, a ASC]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -599,7 +599,7 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)]) -+- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -610,7 +610,7 @@ Calc(select=[CASE((w0$o0 > 0), w0$o1, null:INTEGER) AS EXPR$0, w0$o2 AS EXPR$1]) +- Sort(orderBy=[b ASC, a ASC]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -629,7 +629,7 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)], EXPR$2=[CAST(/(CASE(>(COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 DESC NULLS LAST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 DESC NULLS LAST), null:INTEGER), COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 DESC NULLS LAST))):INTEGER], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)], EXPR$4=[MIN($0) OVER (PARTITION BY $1 ORDER BY $0 DESC NULLS LAST)]) -+- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -644,7 +644,7 @@ Calc(select=[CASE((w0$o0 > 0), w0$o1, null:INTEGER) AS EXPR$0, w0$o2 AS EXPR$1, +- Sort(orderBy=[b ASC, a ASC]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -660,7 +660,7 @@ FROM MyTable @@ -674,7 +674,7 @@ Calc(select=[w0$o0 AS $0, w1$o0 AS $1]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[b ASC, a ASC]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -691,7 +691,7 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), null:INTEGER)], EXPR$1=[MIN($0) OVER (PARTITION BY $1)], EXPR$2=[MAX($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)]) -+- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -706,7 +706,7 @@ Calc(select=[CASE((w0$o0 > 0), w0$o1, null:INTEGER) AS EXPR$0, w1$o0 AS EXPR$1, +- Sort(orderBy=[b ASC]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -722,7 +722,7 @@ FROM MyTable @@ -734,7 +734,7 @@ Calc(select=[a, w0$o0 AS $1, w1$o0 AS $2]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[b ASC, c ASC, a DESC]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.xml index 1ccd27b5af3d4..9320e5fdb573b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.xml @@ -24,7 +24,7 @@ limitations under the License. @@ -34,7 +34,7 @@ SortAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1]) +- Sort(orderBy=[b ASC]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[b, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -46,7 +46,7 @@ SortAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1]) @@ -56,7 +56,7 @@ SortAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1]) +- Sort(orderBy=[b ASC]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[b, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -68,7 +68,7 @@ SortAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1]) @@ -78,7 +78,7 @@ SortAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1]) +- Sort(orderBy=[b ASC]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[b, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -90,7 +90,7 @@ SortAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1]) @@ -99,7 +99,7 @@ SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final +- Exchange(distribution=[single]) +- LocalSortAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3]) +- Calc(select=[a, b, c], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -111,7 +111,7 @@ SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final @@ -119,7 +119,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($ SortAggregate(isMerge=[false], select=[AVG(a) AS EXPR$0, SUM(b) AS EXPR$1, COUNT(c) AS EXPR$2]) +- Exchange(distribution=[single]) +- Calc(select=[a, b, c], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -131,7 +131,7 @@ SortAggregate(isMerge=[false], select=[AVG(a) AS EXPR$0, SUM(b) AS EXPR$1, COUNT @@ -140,7 +140,7 @@ SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final +- Exchange(distribution=[single]) +- LocalSortAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3]) +- Calc(select=[a, b, c], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -153,7 +153,7 @@ SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($2)], EXPR$3=[SUM($3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[$2._1]) +- LogicalFilter(condition=[=($0, 1)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -162,7 +162,7 @@ SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final +- Exchange(distribution=[single]) +- LocalSortAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3, Partial_SUM($f3) AS sum$4]) +- Calc(select=[CAST(1 AS INTEGER) AS a, b, c, c._1 AS $f3], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c]) ]]> @@ -175,7 +175,7 @@ SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($2)], EXPR$3=[SUM($3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[$2._1]) +- LogicalFilter(condition=[=($0, 1)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -183,7 +183,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($ SortAggregate(isMerge=[false], select=[AVG(a) AS EXPR$0, SUM(b) AS EXPR$1, COUNT(c) AS EXPR$2, SUM($f3) AS EXPR$3]) +- Exchange(distribution=[single]) +- Calc(select=[CAST(1 AS INTEGER) AS a, b, c, c._1 AS $f3], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c]) ]]> @@ -196,7 +196,7 @@ SortAggregate(isMerge=[false], select=[AVG(a) AS EXPR$0, SUM(b) AS EXPR$1, COUNT LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($2)], EXPR$3=[SUM($3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[$2._1]) +- LogicalFilter(condition=[=($0, 1)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -205,7 +205,7 @@ SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final +- Exchange(distribution=[single]) +- LocalSortAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3, Partial_SUM($f3) AS sum$4]) +- Calc(select=[CAST(1 AS INTEGER) AS a, b, c, c._1 AS $f3], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c]) ]]> @@ -217,7 +217,7 @@ SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final @@ -230,7 +230,7 @@ SortAggregate(isMerge=[true], groupBy=[a, b], select=[a, b]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC, b ASC]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -242,7 +242,7 @@ SortAggregate(isMerge=[true], groupBy=[a, b], select=[a, b]) @@ -252,7 +252,7 @@ SortAggregate(isMerge=[false], groupBy=[a, b], select=[a, b]) +- Sort(orderBy=[a ASC, b ASC]) +- Exchange(distribution=[hash[a, b]]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -264,7 +264,7 @@ SortAggregate(isMerge=[false], groupBy=[a, b], select=[a, b]) @@ -277,7 +277,7 @@ SortAggregate(isMerge=[true], groupBy=[a, b], select=[a, b]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC, b ASC]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -288,7 +288,7 @@ SortAggregate(isMerge=[true], groupBy=[a, b], select=[a, b]) @@ -296,7 +296,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($ SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final_SUM(sum$2) AS EXPR$1, Final_COUNT(count$3) AS EXPR$2]) +- Exchange(distribution=[single]) +- LocalSortAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -307,14 +307,14 @@ SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final @@ -325,7 +325,7 @@ SortAggregate(isMerge=[false], select=[AVG(a) AS EXPR$0, SUM(b) AS EXPR$1, COUNT @@ -333,7 +333,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($ SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final_SUM(sum$2) AS EXPR$1, Final_COUNT(count$3) AS EXPR$2]) +- Exchange(distribution=[single]) +- LocalSortAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -360,7 +360,7 @@ SELECT @@ -369,7 +369,7 @@ SortAggregate(isMerge=[true], select=[Final_COUNT(APPROXIMATE DISTINCT EXPR$0) A +- Exchange(distribution=[single]) +- LocalSortAggregate(select=[Partial_COUNT(APPROXIMATE DISTINCT byte) AS EXPR$0, Partial_COUNT(APPROXIMATE DISTINCT short) AS EXPR$1, Partial_COUNT(APPROXIMATE DISTINCT int) AS EXPR$2, Partial_COUNT(APPROXIMATE DISTINCT long) AS EXPR$3, Partial_COUNT(APPROXIMATE DISTINCT float) AS EXPR$4, Partial_COUNT(APPROXIMATE DISTINCT double) AS EXPR$5, Partial_COUNT(APPROXIMATE DISTINCT string) AS EXPR$6, Partial_COUNT(APPROXIMATE DISTINCT date) AS EXPR$7, Partial_COUNT(APPROXIMATE DISTINCT time) AS EXPR$8, Partial_COUNT(APPROXIMATE DISTINCT timestamp) AS EXPR$9, Partial_COUNT(APPROXIMATE DISTINCT decimal3020) AS EXPR$10, Partial_COUNT(APPROXIMATE DISTINCT decimal105) AS EXPR$11]) +- Calc(select=[byte, short, int, long, float, double, string, date, time, timestamp, decimal3020, decimal105]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]) ]]> @@ -396,7 +396,7 @@ SELECT @@ -404,7 +404,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT(APPROXIMATE DISTINCT $0)], EXPR$1=[CO SortAggregate(isMerge=[false], select=[COUNT(APPROXIMATE DISTINCT byte) AS EXPR$0, COUNT(APPROXIMATE DISTINCT short) AS EXPR$1, COUNT(APPROXIMATE DISTINCT int) AS EXPR$2, COUNT(APPROXIMATE DISTINCT long) AS EXPR$3, COUNT(APPROXIMATE DISTINCT float) AS EXPR$4, COUNT(APPROXIMATE DISTINCT double) AS EXPR$5, COUNT(APPROXIMATE DISTINCT string) AS EXPR$6, COUNT(APPROXIMATE DISTINCT date) AS EXPR$7, COUNT(APPROXIMATE DISTINCT time) AS EXPR$8, COUNT(APPROXIMATE DISTINCT timestamp) AS EXPR$9, COUNT(APPROXIMATE DISTINCT decimal3020) AS EXPR$10, COUNT(APPROXIMATE DISTINCT decimal105) AS EXPR$11]) +- Exchange(distribution=[single]) +- Calc(select=[byte, short, int, long, float, double, string, date, time, timestamp, decimal3020, decimal105]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]) ]]> @@ -431,7 +431,7 @@ SELECT @@ -440,7 +440,7 @@ SortAggregate(isMerge=[true], select=[Final_COUNT(APPROXIMATE DISTINCT EXPR$0) A +- Exchange(distribution=[single]) +- LocalSortAggregate(select=[Partial_COUNT(APPROXIMATE DISTINCT byte) AS EXPR$0, Partial_COUNT(APPROXIMATE DISTINCT short) AS EXPR$1, Partial_COUNT(APPROXIMATE DISTINCT int) AS EXPR$2, Partial_COUNT(APPROXIMATE DISTINCT long) AS EXPR$3, Partial_COUNT(APPROXIMATE DISTINCT float) AS EXPR$4, Partial_COUNT(APPROXIMATE DISTINCT double) AS EXPR$5, Partial_COUNT(APPROXIMATE DISTINCT string) AS EXPR$6, Partial_COUNT(APPROXIMATE DISTINCT date) AS EXPR$7, Partial_COUNT(APPROXIMATE DISTINCT time) AS EXPR$8, Partial_COUNT(APPROXIMATE DISTINCT timestamp) AS EXPR$9, Partial_COUNT(APPROXIMATE DISTINCT decimal3020) AS EXPR$10, Partial_COUNT(APPROXIMATE DISTINCT decimal105) AS EXPR$11]) +- Calc(select=[byte, short, int, long, float, double, string, date, time, timestamp, decimal3020, decimal105]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]) ]]> @@ -462,7 +462,7 @@ FROM MyTable @@ -471,7 +471,7 @@ SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT sum$0, BIGINT count$1, BIGINT sum$2, BIGINT count$3, BIGINT sum$4, BIGINT count$5, BIGINT sum$6, BIGINT count$7, DOUBLE sum$8, BIGINT count$9, DOUBLE sum$10, BIGINT count$11, DECIMAL(38, 20) sum$12, BIGINT count$13, DECIMAL(38, 5) sum$14, BIGINT count$15)] +- LocalSortAggregate(select=[Partial_AVG(byte) AS (sum$0, count$1), Partial_AVG(short) AS (sum$2, count$3), Partial_AVG(int) AS (sum$4, count$5), Partial_AVG(long) AS (sum$6, count$7), Partial_AVG(float) AS (sum$8, count$9), Partial_AVG(double) AS (sum$10, count$11), Partial_AVG(decimal3020) AS (sum$12, count$13), Partial_AVG(decimal105) AS (sum$14, count$15)]), rowType=[RecordType(BIGINT sum$0, BIGINT count$1, BIGINT sum$2, BIGINT count$3, BIGINT sum$4, BIGINT count$5, BIGINT sum$6, BIGINT count$7, DOUBLE sum$8, BIGINT count$9, DOUBLE sum$10, BIGINT count$11, DECIMAL(38, 20) sum$12, BIGINT count$13, DECIMAL(38, 5) sum$14, BIGINT count$15)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -493,7 +493,7 @@ FROM MyTable @@ -501,7 +501,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[AVG($1)], EXPR$2=[AVG($2) SortAggregate(isMerge=[false], select=[AVG(byte) AS EXPR$0, AVG(short) AS EXPR$1, AVG(int) AS EXPR$2, AVG(long) AS EXPR$3, AVG(float) AS EXPR$4, AVG(double) AS EXPR$5, AVG(decimal3020) AS EXPR$6, AVG(decimal105) AS EXPR$7]), rowType=[RecordType(TINYINT EXPR$0, SMALLINT EXPR$1, INTEGER EXPR$2, BIGINT EXPR$3, FLOAT EXPR$4, DOUBLE EXPR$5, DECIMAL(38, 20) EXPR$6, DECIMAL(38, 6) EXPR$7)] +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -523,7 +523,7 @@ FROM MyTable @@ -532,7 +532,7 @@ SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT sum$0, BIGINT count$1, BIGINT sum$2, BIGINT count$3, BIGINT sum$4, BIGINT count$5, BIGINT sum$6, BIGINT count$7, DOUBLE sum$8, BIGINT count$9, DOUBLE sum$10, BIGINT count$11, DECIMAL(38, 20) sum$12, BIGINT count$13, DECIMAL(38, 5) sum$14, BIGINT count$15)] +- LocalSortAggregate(select=[Partial_AVG(byte) AS (sum$0, count$1), Partial_AVG(short) AS (sum$2, count$3), Partial_AVG(int) AS (sum$4, count$5), Partial_AVG(long) AS (sum$6, count$7), Partial_AVG(float) AS (sum$8, count$9), Partial_AVG(double) AS (sum$10, count$11), Partial_AVG(decimal3020) AS (sum$12, count$13), Partial_AVG(decimal105) AS (sum$14, count$15)]), rowType=[RecordType(BIGINT sum$0, BIGINT count$1, BIGINT sum$2, BIGINT count$3, BIGINT sum$4, BIGINT count$5, BIGINT sum$6, BIGINT count$7, DOUBLE sum$8, BIGINT count$9, DOUBLE sum$10, BIGINT count$11, DECIMAL(38, 20) sum$12, BIGINT count$13, DECIMAL(38, 5) sum$14, BIGINT count$15)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -559,7 +559,7 @@ FROM MyTable @@ -567,7 +567,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT($0)], EXPR$1=[COUNT($1)], EXPR$2=[COU SortAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_COUNT(count$1) AS EXPR$1, Final_COUNT(count$2) AS EXPR$2, Final_COUNT(count$3) AS EXPR$3, Final_COUNT(count$4) AS EXPR$4, Final_COUNT(count$5) AS EXPR$5, Final_COUNT(count$6) AS EXPR$6, Final_COUNT(count$7) AS EXPR$7, Final_COUNT(count$8) AS EXPR$8, Final_COUNT(count$9) AS EXPR$9, Final_COUNT(count$10) AS EXPR$10, Final_COUNT(count$11) AS EXPR$11, Final_COUNT(count$12) AS EXPR$12]), rowType=[RecordType(BIGINT EXPR$0, BIGINT EXPR$1, BIGINT EXPR$2, BIGINT EXPR$3, BIGINT EXPR$4, BIGINT EXPR$5, BIGINT EXPR$6, BIGINT EXPR$7, BIGINT EXPR$8, BIGINT EXPR$9, BIGINT EXPR$10, BIGINT EXPR$11, BIGINT EXPR$12)] +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count$0, BIGINT count$1, BIGINT count$2, BIGINT count$3, BIGINT count$4, BIGINT count$5, BIGINT count$6, BIGINT count$7, BIGINT count$8, BIGINT count$9, BIGINT count$10, BIGINT count$11, BIGINT count$12)] +- LocalSortAggregate(select=[Partial_COUNT(byte) AS count$0, Partial_COUNT(short) AS count$1, Partial_COUNT(int) AS count$2, Partial_COUNT(long) AS count$3, Partial_COUNT(float) AS count$4, Partial_COUNT(double) AS count$5, Partial_COUNT(decimal3020) AS count$6, Partial_COUNT(decimal105) AS count$7, Partial_COUNT(boolean) AS count$8, Partial_COUNT(date) AS count$9, Partial_COUNT(time) AS count$10, Partial_COUNT(timestamp) AS count$11, Partial_COUNT(string) AS count$12]), rowType=[RecordType(BIGINT count$0, BIGINT count$1, BIGINT count$2, BIGINT count$3, BIGINT count$4, BIGINT count$5, BIGINT count$6, BIGINT count$7, BIGINT count$8, BIGINT count$9, BIGINT count$10, BIGINT count$11, BIGINT count$12)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -594,14 +594,14 @@ FROM MyTable @@ -628,7 +628,7 @@ FROM MyTable @@ -636,7 +636,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT($0)], EXPR$1=[COUNT($1)], EXPR$2=[COU SortAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_COUNT(count$1) AS EXPR$1, Final_COUNT(count$2) AS EXPR$2, Final_COUNT(count$3) AS EXPR$3, Final_COUNT(count$4) AS EXPR$4, Final_COUNT(count$5) AS EXPR$5, Final_COUNT(count$6) AS EXPR$6, Final_COUNT(count$7) AS EXPR$7, Final_COUNT(count$8) AS EXPR$8, Final_COUNT(count$9) AS EXPR$9, Final_COUNT(count$10) AS EXPR$10, Final_COUNT(count$11) AS EXPR$11, Final_COUNT(count$12) AS EXPR$12]), rowType=[RecordType(BIGINT EXPR$0, BIGINT EXPR$1, BIGINT EXPR$2, BIGINT EXPR$3, BIGINT EXPR$4, BIGINT EXPR$5, BIGINT EXPR$6, BIGINT EXPR$7, BIGINT EXPR$8, BIGINT EXPR$9, BIGINT EXPR$10, BIGINT EXPR$11, BIGINT EXPR$12)] +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count$0, BIGINT count$1, BIGINT count$2, BIGINT count$3, BIGINT count$4, BIGINT count$5, BIGINT count$6, BIGINT count$7, BIGINT count$8, BIGINT count$9, BIGINT count$10, BIGINT count$11, BIGINT count$12)] +- LocalSortAggregate(select=[Partial_COUNT(byte) AS count$0, Partial_COUNT(short) AS count$1, Partial_COUNT(int) AS count$2, Partial_COUNT(long) AS count$3, Partial_COUNT(float) AS count$4, Partial_COUNT(double) AS count$5, Partial_COUNT(decimal3020) AS count$6, Partial_COUNT(decimal105) AS count$7, Partial_COUNT(boolean) AS count$8, Partial_COUNT(date) AS count$9, Partial_COUNT(time) AS count$10, Partial_COUNT(timestamp) AS count$11, Partial_COUNT(string) AS count$12]), rowType=[RecordType(BIGINT count$0, BIGINT count$1, BIGINT count$2, BIGINT count$3, BIGINT count$4, BIGINT count$5, BIGINT count$6, BIGINT count$7, BIGINT count$8, BIGINT count$9, BIGINT count$10, BIGINT count$11, BIGINT count$12)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -648,7 +648,7 @@ SortAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_COUN @@ -657,7 +657,7 @@ SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count1$0)] +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0]), rowType=[RecordType(BIGINT count1$0)] +- Calc(select=[0 AS $f0]), rowType=[RecordType(INTEGER $f0)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -669,7 +669,7 @@ SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType @@ -677,7 +677,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]), rowType=[RecordType(BIGINT EXPR$ SortAggregate(isMerge=[false], select=[COUNT(*) AS EXPR$0]), rowType=[RecordType(BIGINT EXPR$0)] +- Exchange(distribution=[single]), rowType=[RecordType(INTEGER $f0)] +- Calc(select=[0 AS $f0]), rowType=[RecordType(INTEGER $f0)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -689,7 +689,7 @@ SortAggregate(isMerge=[false], select=[COUNT(*) AS EXPR$0]), rowType=[RecordType @@ -698,7 +698,7 @@ SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count1$0)] +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0]), rowType=[RecordType(BIGINT count1$0)] +- Calc(select=[0 AS $f0]), rowType=[RecordType(INTEGER $f0)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -767,7 +767,7 @@ SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType @@ -779,7 +779,7 @@ SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1 +- LocalSortAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_COUNT(c) AS count$1]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -790,7 +790,7 @@ SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1 @@ -799,7 +799,7 @@ SortAggregate(isMerge=[false], groupBy=[a], select=[a, SUM(b) AS EXPR$1, COUNT(c +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC]) +- Exchange(distribution=[hash[a]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -810,7 +810,7 @@ SortAggregate(isMerge=[false], groupBy=[a], select=[a, SUM(b) AS EXPR$1, COUNT(c @@ -822,7 +822,7 @@ SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1 +- LocalSortAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_COUNT(c) AS count$1]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -834,7 +834,7 @@ SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1 @@ -847,7 +847,7 @@ SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1 +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC]) +- Calc(select=[a, b, c], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -859,7 +859,7 @@ SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1 @@ -869,7 +869,7 @@ SortAggregate(isMerge=[false], groupBy=[a], select=[a, SUM(b) AS EXPR$1, COUNT(c +- Sort(orderBy=[a ASC]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, c], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -881,7 +881,7 @@ SortAggregate(isMerge=[false], groupBy=[a], select=[a, SUM(b) AS EXPR$1, COUNT(c @@ -894,7 +894,7 @@ SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1 +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC]) +- Calc(select=[a, b, c], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -909,7 +909,7 @@ SELECT a, MAX(b), c FROM (SELECT a, 'test' AS c, b FROM MyTable1) t GROUP BY a, LogicalProject(a=[$0], EXPR$1=[$2], c=[$1]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[MAX($2)]) +- LogicalProject(a=[$0], c=[_UTF-16LE'test'], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -923,7 +923,7 @@ Calc(select=[a, EXPR$1, 'test' AS c]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -938,7 +938,7 @@ SELECT a, MAX(b), c FROM (SELECT a, 'test' AS c, b FROM MyTable1) t GROUP BY a, LogicalProject(a=[$0], EXPR$1=[$2], c=[$1]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[MAX($2)]) +- LogicalProject(a=[$0], c=[_UTF-16LE'test'], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -949,7 +949,7 @@ Calc(select=[a, EXPR$1, 'test' AS c]) +- Sort(orderBy=[a ASC]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -964,7 +964,7 @@ SELECT a, MAX(b), c FROM (SELECT a, 'test' AS c, b FROM MyTable1) t GROUP BY a, LogicalProject(a=[$0], EXPR$1=[$2], c=[$1]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[MAX($2)]) +- LogicalProject(a=[$0], c=[_UTF-16LE'test'], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -978,7 +978,7 @@ Calc(select=[a, EXPR$1, 'test' AS c]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[a ASC]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -1004,7 +1004,7 @@ FROM MyTable @@ -1013,7 +1013,7 @@ SortAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0, Final_MAX(max$ +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT max$0, SMALLINT max$1, INTEGER max$2, BIGINT max$3, FLOAT max$4, DOUBLE max$5, DECIMAL(30, 20) max$6, DECIMAL(10, 5) max$7, BOOLEAN max$8, DATE max$9, TIME(0) max$10, TIMESTAMP(3) max$11)] +- LocalSortAggregate(select=[Partial_MAX(byte) AS max$0, Partial_MAX(short) AS max$1, Partial_MAX(int) AS max$2, Partial_MAX(long) AS max$3, Partial_MAX(float) AS max$4, Partial_MAX(double) AS max$5, Partial_MAX(decimal3020) AS max$6, Partial_MAX(decimal105) AS max$7, Partial_MAX(boolean) AS max$8, Partial_MAX(date) AS max$9, Partial_MAX(time) AS max$10, Partial_MAX(timestamp) AS max$11]), rowType=[RecordType(TINYINT max$0, SMALLINT max$1, INTEGER max$2, BIGINT max$3, FLOAT max$4, DOUBLE max$5, DECIMAL(30, 20) max$6, DECIMAL(10, 5) max$7, BOOLEAN max$8, DATE max$9, TIME(0) max$10, TIMESTAMP(3) max$11)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -1039,7 +1039,7 @@ FROM MyTable @@ -1047,7 +1047,7 @@ LogicalAggregate(group=[{}], EXPR$0=[MAX($0)], EXPR$1=[MAX($1)], EXPR$2=[MAX($2) SortAggregate(isMerge=[false], select=[MAX(byte) AS EXPR$0, MAX(short) AS EXPR$1, MAX(int) AS EXPR$2, MAX(long) AS EXPR$3, MAX(float) AS EXPR$4, MAX(double) AS EXPR$5, MAX(decimal3020) AS EXPR$6, MAX(decimal105) AS EXPR$7, MAX(boolean) AS EXPR$8, MAX(date) AS EXPR$9, MAX(time) AS EXPR$10, MAX(timestamp) AS EXPR$11]), rowType=[RecordType(TINYINT EXPR$0, SMALLINT EXPR$1, INTEGER EXPR$2, BIGINT EXPR$3, FLOAT EXPR$4, DOUBLE EXPR$5, DECIMAL(30, 20) EXPR$6, DECIMAL(10, 5) EXPR$7, BOOLEAN EXPR$8, DATE EXPR$9, TIME(0) EXPR$10, TIMESTAMP(3) EXPR$11)] +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -1073,7 +1073,7 @@ FROM MyTable @@ -1082,7 +1082,7 @@ SortAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0, Final_MAX(max$ +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT max$0, SMALLINT max$1, INTEGER max$2, BIGINT max$3, FLOAT max$4, DOUBLE max$5, DECIMAL(30, 20) max$6, DECIMAL(10, 5) max$7, BOOLEAN max$8, DATE max$9, TIME(0) max$10, TIMESTAMP(3) max$11)] +- LocalSortAggregate(select=[Partial_MAX(byte) AS max$0, Partial_MAX(short) AS max$1, Partial_MAX(int) AS max$2, Partial_MAX(long) AS max$3, Partial_MAX(float) AS max$4, Partial_MAX(double) AS max$5, Partial_MAX(decimal3020) AS max$6, Partial_MAX(decimal105) AS max$7, Partial_MAX(boolean) AS max$8, Partial_MAX(date) AS max$9, Partial_MAX(time) AS max$10, Partial_MAX(timestamp) AS max$11]), rowType=[RecordType(TINYINT max$0, SMALLINT max$1, INTEGER max$2, BIGINT max$3, FLOAT max$4, DOUBLE max$5, DECIMAL(30, 20) max$6, DECIMAL(10, 5) max$7, BOOLEAN max$8, DATE max$9, TIME(0) max$10, TIMESTAMP(3) max$11)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -1094,7 +1094,7 @@ SortAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0, Final_MAX(max$ @@ -1103,7 +1103,7 @@ SortAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0]), rowType=[Rec +- Exchange(distribution=[single]), rowType=[RecordType(VARCHAR(2147483647) max$0)] +- LocalSortAggregate(select=[Partial_MAX(string) AS max$0]), rowType=[RecordType(VARCHAR(2147483647) max$0)] +- Calc(select=[string]), rowType=[RecordType(VARCHAR(2147483647) string)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -1115,7 +1115,7 @@ SortAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0]), rowType=[Rec @@ -1123,7 +1123,7 @@ LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]), rowType=[RecordType(VARCHAR(2147 SortAggregate(isMerge=[false], select=[MAX(string) AS EXPR$0]), rowType=[RecordType(VARCHAR(2147483647) EXPR$0)] +- Exchange(distribution=[single]), rowType=[RecordType(VARCHAR(2147483647) string)] +- Calc(select=[string]), rowType=[RecordType(VARCHAR(2147483647) string)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -1135,7 +1135,7 @@ SortAggregate(isMerge=[false], select=[MAX(string) AS EXPR$0]), rowType=[RecordT @@ -1144,7 +1144,7 @@ SortAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0]), rowType=[Rec +- Exchange(distribution=[single]), rowType=[RecordType(VARCHAR(2147483647) max$0)] +- LocalSortAggregate(select=[Partial_MAX(string) AS max$0]), rowType=[RecordType(VARCHAR(2147483647) max$0)] +- Calc(select=[string]), rowType=[RecordType(VARCHAR(2147483647) string)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -1170,7 +1170,7 @@ FROM MyTable @@ -1179,7 +1179,7 @@ SortAggregate(isMerge=[true], select=[Final_MIN(min$0) AS EXPR$0, Final_MIN(min$ +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT min$0, SMALLINT min$1, INTEGER min$2, BIGINT min$3, FLOAT min$4, DOUBLE min$5, DECIMAL(30, 20) min$6, DECIMAL(10, 5) min$7, BOOLEAN min$8, DATE min$9, TIME(0) min$10, TIMESTAMP(3) min$11)] +- LocalSortAggregate(select=[Partial_MIN(byte) AS min$0, Partial_MIN(short) AS min$1, Partial_MIN(int) AS min$2, Partial_MIN(long) AS min$3, Partial_MIN(float) AS min$4, Partial_MIN(double) AS min$5, Partial_MIN(decimal3020) AS min$6, Partial_MIN(decimal105) AS min$7, Partial_MIN(boolean) AS min$8, Partial_MIN(date) AS min$9, Partial_MIN(time) AS min$10, Partial_MIN(timestamp) AS min$11]), rowType=[RecordType(TINYINT min$0, SMALLINT min$1, INTEGER min$2, BIGINT min$3, FLOAT min$4, DOUBLE min$5, DECIMAL(30, 20) min$6, DECIMAL(10, 5) min$7, BOOLEAN min$8, DATE min$9, TIME(0) min$10, TIMESTAMP(3) min$11)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -1205,7 +1205,7 @@ FROM MyTable @@ -1213,7 +1213,7 @@ LogicalAggregate(group=[{}], EXPR$0=[MIN($0)], EXPR$1=[MIN($1)], EXPR$2=[MIN($2) SortAggregate(isMerge=[false], select=[MIN(byte) AS EXPR$0, MIN(short) AS EXPR$1, MIN(int) AS EXPR$2, MIN(long) AS EXPR$3, MIN(float) AS EXPR$4, MIN(double) AS EXPR$5, MIN(decimal3020) AS EXPR$6, MIN(decimal105) AS EXPR$7, MIN(boolean) AS EXPR$8, MIN(date) AS EXPR$9, MIN(time) AS EXPR$10, MIN(timestamp) AS EXPR$11]), rowType=[RecordType(TINYINT EXPR$0, SMALLINT EXPR$1, INTEGER EXPR$2, BIGINT EXPR$3, FLOAT EXPR$4, DOUBLE EXPR$5, DECIMAL(30, 20) EXPR$6, DECIMAL(10, 5) EXPR$7, BOOLEAN EXPR$8, DATE EXPR$9, TIME(0) EXPR$10, TIMESTAMP(3) EXPR$11)] +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -1239,7 +1239,7 @@ FROM MyTable @@ -1248,7 +1248,7 @@ SortAggregate(isMerge=[true], select=[Final_MIN(min$0) AS EXPR$0, Final_MIN(min$ +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT min$0, SMALLINT min$1, INTEGER min$2, BIGINT min$3, FLOAT min$4, DOUBLE min$5, DECIMAL(30, 20) min$6, DECIMAL(10, 5) min$7, BOOLEAN min$8, DATE min$9, TIME(0) min$10, TIMESTAMP(3) min$11)] +- LocalSortAggregate(select=[Partial_MIN(byte) AS min$0, Partial_MIN(short) AS min$1, Partial_MIN(int) AS min$2, Partial_MIN(long) AS min$3, Partial_MIN(float) AS min$4, Partial_MIN(double) AS min$5, Partial_MIN(decimal3020) AS min$6, Partial_MIN(decimal105) AS min$7, Partial_MIN(boolean) AS min$8, Partial_MIN(date) AS min$9, Partial_MIN(time) AS min$10, Partial_MIN(timestamp) AS min$11]), rowType=[RecordType(TINYINT min$0, SMALLINT min$1, INTEGER min$2, BIGINT min$3, FLOAT min$4, DOUBLE min$5, DECIMAL(30, 20) min$6, DECIMAL(10, 5) min$7, BOOLEAN min$8, DATE min$9, TIME(0) min$10, TIMESTAMP(3) min$11)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -1260,7 +1260,7 @@ SortAggregate(isMerge=[true], select=[Final_MIN(min$0) AS EXPR$0, Final_MIN(min$ @@ -1269,7 +1269,7 @@ SortAggregate(isMerge=[true], select=[Final_MIN(min$0) AS EXPR$0]), rowType=[Rec +- Exchange(distribution=[single]), rowType=[RecordType(VARCHAR(2147483647) min$0)] +- LocalSortAggregate(select=[Partial_MIN(string) AS min$0]), rowType=[RecordType(VARCHAR(2147483647) min$0)] +- Calc(select=[string]), rowType=[RecordType(VARCHAR(2147483647) string)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -1281,7 +1281,7 @@ SortAggregate(isMerge=[true], select=[Final_MIN(min$0) AS EXPR$0]), rowType=[Rec @@ -1289,7 +1289,7 @@ LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]), rowType=[RecordType(VARCHAR(2147 SortAggregate(isMerge=[false], select=[MIN(string) AS EXPR$0]), rowType=[RecordType(VARCHAR(2147483647) EXPR$0)] +- Exchange(distribution=[single]), rowType=[RecordType(VARCHAR(2147483647) string)] +- Calc(select=[string]), rowType=[RecordType(VARCHAR(2147483647) string)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -1301,7 +1301,7 @@ SortAggregate(isMerge=[false], select=[MIN(string) AS EXPR$0]), rowType=[RecordT @@ -1310,7 +1310,7 @@ SortAggregate(isMerge=[true], select=[Final_MIN(min$0) AS EXPR$0]), rowType=[Rec +- Exchange(distribution=[single]), rowType=[RecordType(VARCHAR(2147483647) min$0)] +- LocalSortAggregate(select=[Partial_MIN(string) AS min$0]), rowType=[RecordType(VARCHAR(2147483647) min$0)] +- Calc(select=[string]), rowType=[RecordType(VARCHAR(2147483647) string)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -1322,7 +1322,7 @@ SortAggregate(isMerge=[true], select=[Final_MIN(min$0) AS EXPR$0]), rowType=[Rec @@ -1332,7 +1332,7 @@ SortAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1]) +- Sort(orderBy=[b ASC]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[b, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -1344,7 +1344,7 @@ SortAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1]) @@ -1354,7 +1354,7 @@ SortAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1]) +- Sort(orderBy=[b ASC]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[b, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -1366,7 +1366,7 @@ SortAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1]) @@ -1376,7 +1376,7 @@ SortAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1]) +- Sort(orderBy=[b ASC]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[b, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -1398,7 +1398,7 @@ FROM MyTable @@ -1407,7 +1407,7 @@ SortAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS EXPR$0, Final_SUM(sum$ +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT sum$0, SMALLINT sum$1, INTEGER sum$2, BIGINT sum$3, FLOAT sum$4, DOUBLE sum$5, DECIMAL(38, 20) sum$6, DECIMAL(38, 5) sum$7)] +- LocalSortAggregate(select=[Partial_SUM(byte) AS sum$0, Partial_SUM(short) AS sum$1, Partial_SUM(int) AS sum$2, Partial_SUM(long) AS sum$3, Partial_SUM(float) AS sum$4, Partial_SUM(double) AS sum$5, Partial_SUM(decimal3020) AS sum$6, Partial_SUM(decimal105) AS sum$7]), rowType=[RecordType(TINYINT sum$0, SMALLINT sum$1, INTEGER sum$2, BIGINT sum$3, FLOAT sum$4, DOUBLE sum$5, DECIMAL(38, 20) sum$6, DECIMAL(38, 5) sum$7)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -1429,7 +1429,7 @@ FROM MyTable @@ -1437,7 +1437,7 @@ LogicalAggregate(group=[{}], EXPR$0=[SUM($0)], EXPR$1=[SUM($1)], EXPR$2=[SUM($2) SortAggregate(isMerge=[false], select=[SUM(byte) AS EXPR$0, SUM(short) AS EXPR$1, SUM(int) AS EXPR$2, SUM(long) AS EXPR$3, SUM(float) AS EXPR$4, SUM(double) AS EXPR$5, SUM(decimal3020) AS EXPR$6, SUM(decimal105) AS EXPR$7]), rowType=[RecordType(TINYINT EXPR$0, SMALLINT EXPR$1, INTEGER EXPR$2, BIGINT EXPR$3, FLOAT EXPR$4, DOUBLE EXPR$5, DECIMAL(38, 20) EXPR$6, DECIMAL(38, 5) EXPR$7)] +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -1459,7 +1459,7 @@ FROM MyTable @@ -1468,7 +1468,7 @@ SortAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS EXPR$0, Final_SUM(sum$ +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT sum$0, SMALLINT sum$1, INTEGER sum$2, BIGINT sum$3, FLOAT sum$4, DOUBLE sum$5, DECIMAL(38, 20) sum$6, DECIMAL(38, 5) sum$7)] +- LocalSortAggregate(select=[Partial_SUM(byte) AS sum$0, Partial_SUM(short) AS sum$1, Partial_SUM(int) AS sum$2, Partial_SUM(long) AS sum$3, Partial_SUM(float) AS sum$4, Partial_SUM(double) AS sum$5, Partial_SUM(decimal3020) AS sum$6, Partial_SUM(decimal105) AS sum$7]), rowType=[RecordType(TINYINT sum$0, SMALLINT sum$1, INTEGER sum$2, BIGINT sum$3, FLOAT sum$4, DOUBLE sum$5, DECIMAL(38, 20) sum$6, DECIMAL(38, 5) sum$7)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml index 830c71fabe84e..153b792d51b3b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml @@ -24,8 +24,8 @@ limitations under the License. @@ -33,10 +33,10 @@ LogicalProject(c=[$7], g=[$3]) Calc(select=[c, g]) +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[d, g, a, c], isBroadcast=[true], build=[right]) :- Calc(select=[d, g], where=[(d < 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[broadcast]) +- Calc(select=[a, c], where=[(a < 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -48,16 +48,16 @@ Calc(select=[c, g]) @@ -69,18 +69,18 @@ HashJoin(joinType=[LeftOuterJoin], where=[((a = d) AND (d < 2) AND (b < h))], se @@ -93,8 +93,8 @@ HashJoin(joinType=[InnerJoin], where=[((a = d) AND (b < h))], select=[d, e, f, g LogicalProject(c=[$2], g=[$6]) +- LogicalFilter(condition=[=($0, $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -102,10 +102,10 @@ LogicalProject(c=[$2], g=[$6]) Calc(select=[c, g]) +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, c, d, g], isBroadcast=[true], build=[right]) :- Calc(select=[a, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -118,8 +118,8 @@ Calc(select=[c, g]) LogicalProject(c=[$7], g=[$3]) +- LogicalFilter(condition=[AND(=($5, $0), <($0, 2))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -127,10 +127,10 @@ LogicalProject(c=[$7], g=[$3]) Calc(select=[c, g]) +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[d, g, a, c], isBroadcast=[true], build=[right]) :- Calc(select=[d, g], where=[(d < 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[broadcast]) +- Calc(select=[a, c], where=[(a < 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -151,10 +151,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -166,11 +166,11 @@ Calc(select=[a, b, CAST(2 AS INTEGER) AS d, e]) : +- Exchange(distribution=[hash[d]]) : +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) : +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -183,18 +183,18 @@ Calc(select=[a, b, CAST(2 AS INTEGER) AS d, e]) LogicalProject(c=[$2], g=[$6]) +- LogicalFilter(condition=[AND(=($1, $4), =($0, $3))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -206,8 +206,8 @@ Calc(select=[c, g]) @@ -215,9 +215,9 @@ LogicalProject(c=[$7], g=[$3]) Calc(select=[c, g]) +- HashJoin(joinType=[InnerJoin], where=[((a = d) AND (b = e))], select=[d, e, g, a, b, c], isBroadcast=[true], build=[right]) :- Calc(select=[d, e, g]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[broadcast]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -230,8 +230,8 @@ Calc(select=[c, g]) LogicalProject(a=[$0], d=[$3]) +- LogicalFilter(condition=[=(+($0, 1), $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -239,10 +239,10 @@ LogicalProject(a=[$0], d=[$3]) Calc(select=[a, d]) +- HashJoin(joinType=[InnerJoin], where=[($f3 = d)], select=[a, $f3, d], isBroadcast=[true], build=[right]) :- Calc(select=[a, (a + 1) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -284,10 +284,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[true], joinType=[left]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -299,11 +299,11 @@ Calc(select=[CAST(2 AS INTEGER) AS a, b, d, CAST(e AS BIGINT) AS e]) : +- Exchange(distribution=[hash[d]]) : +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) : +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -323,10 +323,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($0, 2), =($3, 1))], joinType=[left]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -338,11 +338,11 @@ MultipleInput(readOrder=[0,1], members=[\nHashJoin(joinType=[LeftOuterJoin], whe : +- Exchange(distribution=[hash[d]]) : +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) : +- Calc(select=[d, e]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -354,8 +354,8 @@ MultipleInput(readOrder=[0,1], members=[\nHashJoin(joinType=[LeftOuterJoin], whe @@ -363,10 +363,10 @@ LogicalProject(c=[$7], g=[$3]) Calc(select=[c, g]) +- HashJoin(joinType=[LeftOuterJoin], where=[((a = d) AND (d < 2))], select=[d, g, a, c], isBroadcast=[true], build=[right]) :- Calc(select=[d, g]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[broadcast]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -378,8 +378,8 @@ Calc(select=[c, g]) @@ -388,9 +388,9 @@ Calc(select=[c, g]) +- HashJoin(joinType=[RightOuterJoin], where=[(b = e)], select=[b, c, e, g], isBroadcast=[true], build=[left]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[b, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Calc(select=[e, g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -410,10 +410,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($2, 2), =($1, 1))], joinType=[inner]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -426,11 +426,11 @@ Calc(select=[a, 1 AS b, CAST(2 AS INTEGER) AS d, 1 AS e]) : +- Exchange(distribution=[hash[d]]) : +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) : +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -443,8 +443,8 @@ Calc(select=[a, 1 AS b, CAST(2 AS INTEGER) AS d, 1 AS e]) LogicalProject(d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(IS NULL($3), <($0, 12))]) +- LogicalJoin(condition=[=($0, $3)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -452,10 +452,10 @@ LogicalProject(d=[$3], e=[$4], f=[$5]) Calc(select=[null:INTEGER AS d, e, f], where=[d IS NULL]) +- HashJoin(joinType=[LeftOuterJoin], where=[(a = d)], select=[a, d, e, f], isBroadcast=[true], build=[right]) :- Calc(select=[a], where=[(a < 12)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, e, f], where=[(d < 12)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -468,8 +468,8 @@ Calc(select=[null:INTEGER AS d, e, f], where=[d IS NULL]) LogicalProject(d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(<($3, 10), <($0, 12))]) +- LogicalJoin(condition=[=($0, $3)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -478,9 +478,9 @@ Calc(select=[d, e, f]) +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d, e, f], isBroadcast=[true], build=[left]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a], where=[(a < 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Calc(select=[d, e, f], where=[(d < 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -493,8 +493,8 @@ Calc(select=[d, e, f]) LogicalProject(d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[=($3, null)]) +- LogicalJoin(condition=[=($0, $3)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -520,10 +520,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[true], joinType=[right]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -535,11 +535,11 @@ Calc(select=[a, CAST(b AS BIGINT) AS b, CAST(2 AS INTEGER) AS d, e]) : +- Exchange(distribution=[hash[d]]) : +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) : +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -559,10 +559,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($2, 2), =($1, 1))], joinType=[right]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -574,11 +574,11 @@ MultipleInput(readOrder=[0,1], members=[\nHashJoin(joinType=[RightOuterJoin], wh : +- Exchange(distribution=[hash[a]]) : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) : +- Calc(select=[a, b]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -590,8 +590,8 @@ MultipleInput(readOrder=[0,1], members=[\nHashJoin(joinType=[RightOuterJoin], wh @@ -599,10 +599,10 @@ LogicalProject(c=[$2], g=[$6]) Calc(select=[c, g]) +- HashJoin(joinType=[LeftOuterJoin], where=[(b = e)], select=[b, c, e, g], isBroadcast=[true], build=[right]) :- Calc(select=[b, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[e, g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -614,8 +614,8 @@ Calc(select=[c, g]) @@ -623,8 +623,8 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7]) HashJoin(joinType=[RightOuterJoin], where=[((a = d) AND (b < h))], select=[d, e, f, g, h, a, b, c], isBroadcast=[true], build=[left]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[d, e, f, g, h], where=[(d < 2)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -636,8 +636,8 @@ HashJoin(joinType=[RightOuterJoin], where=[((a = d) AND (b < h))], select=[d, e, @@ -646,9 +646,9 @@ Calc(select=[c, g]) +- HashJoin(joinType=[RightOuterJoin], where=[(a = d)], select=[d, g, a, c], isBroadcast=[true], build=[left]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[d, g], where=[(d < 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -667,16 +667,16 @@ LogicalProject(k=[$0], v=[$1], k0=[$2], v0=[$3]) +- LogicalJoin(condition=[AND(=($0, $2), >($2, 10))], joinType=[left]) :- LogicalProject(k=[$0], v=[$1]) : +- LogicalFilter(condition=[=($0, 0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, src]]) +- LogicalProject(k=[$0], v=[$1]) +- LogicalFilter(condition=[=($0, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, src]]) ]]> @@ -688,14 +688,14 @@ Calc(select=[CAST(0 AS BIGINT) AS k, v, null:BIGINT AS k0, null:VARCHAR(21474836 @@ -707,14 +707,14 @@ Calc(select=[null:INTEGER AS d, null:BIGINT AS e, null:INTEGER AS f, null:VARCHA diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml index 441f86e8fa99b..0f8b17670d4eb 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml @@ -24,28 +24,28 @@ limitations under the License. @@ -60,22 +60,22 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$0=[MAX($4)]) LogicalProject(d=[$0], $f1=[true], f=[$2], $f3=[1], e=[$1]) LogicalFilter(condition=[AND(=($cor0.b, $1), <($0, 100), =($cor0.c, $2))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -89,20 +89,20 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=($cor1.a, $0)]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })], variablesSet=[[$cor1]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -117,22 +117,22 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalAggregate(group=[{0}]) LogicalProject(e=[$1]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -145,18 +145,18 @@ HashJoin(joinType=[LeftSemiJoin], where=[(c = f)], select=[a, b, c], isBroadcast LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -168,9 +168,9 @@ HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], isBroadcast ($1, 10), NOT(OR(LIKE($2, _UTF-16LE'abc'), NOT(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) +LogicalTableScan(table=[[default_catalog, default_database, r]]) })))))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -178,14 +178,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl Calc(select=[(a + 10) AS EXPR$0, c]) +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, c], build=[right], singleRowJoin=[true]) :- Calc(select=[a, c], where=[((b > 10) AND NOT(LIKE(c, 'abc')))]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[], where=[m IS NOT NULL]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -198,19 +198,19 @@ Calc(select=[(a + 10) AS EXPR$0, c]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) +LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -239,22 +239,22 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], build=[r @@ -266,23 +266,23 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], build=[r ($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[], where=[m IS NOT NULL]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -297,19 +297,19 @@ LogicalProject(a=[$0]) LogicalUnion(all=[false]) LogicalProject(e=[$1], f=[$2]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1], k=[$2]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Calc(select=[j, k], where=[(i < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -336,25 +336,25 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a], build=[right], LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -367,24 +367,24 @@ MultipleInput(readOrder=[0,1,0], members=[\nHashJoin(joinType=[LeftSemiJoin], wh LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -400,22 +400,22 @@ LogicalProject(EXPR$0=[$1]) LogicalAggregate(group=[{0}], EXPR$0=[MAX($1)]) LogicalProject(f=[$2], e=[$1]) LogicalFilter(condition=[AND(=($cor0.c, $2), <($0, 3))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -431,21 +431,21 @@ LogicalProject(EXPR$0=[$4], d=[$0]) LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$0=[COUNT()]) LogicalProject(d=[$0], $f1=[true], e=[$1], $f3=[1]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -459,9 +459,9 @@ LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) +- LogicalFilter(condition=[IN(SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=(+($cor0.b, 1), $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -469,10 +469,10 @@ LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)]) Calc(select=[(a + 10) AS EXPR$0, c]) +- HashJoin(joinType=[LeftSemiJoin], where=[(($f3 = EXPR$0) AND ($f4 = e))], select=[a, c, $f3, $f4], isBroadcast=[true], build=[right]) :- Calc(select=[a, c, SUBSTRING(c, 1, 5) AS $f3, (b + 1) AS $f4]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[SUBSTRING(f, 1, 5) AS EXPR$0, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -486,19 +486,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), NOT(LIKE($2, _UTF-16LE'abc')), IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10) AND NOT(LIKE(c, 'abc')))]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -514,27 +514,27 @@ LogicalProject(e=[$0]) LogicalJoin(condition=[=($0, $1)], joinType=[inner]) LogicalProject(e=[$1]) LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1]) LogicalFilter(condition=[AND(=($cor0.c, $2), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -550,25 +550,25 @@ LogicalProject(j=[$3]) LogicalJoin(condition=[=($1, $4)], joinType=[left]) LogicalProject(e=[$1], f=[$2]) LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -582,12 +582,12 @@ LogicalProject(c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalJoin(condition=[=($0, $3)], joinType=[right]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0]) LogicalFilter(condition=[AND(=($cor0.c, $2), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -595,15 +595,15 @@ LogicalProject(d=[$0]) Calc(select=[c]) +- HashJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (c = k))], select=[a, c], isBroadcast=[true], build=[right]) :- Calc(select=[a, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, k]) +- HashJoin(joinType=[RightOuterJoin], where=[(d = i)], select=[d, i, k], isBroadcast=[true], build=[left]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[d]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Calc(select=[i, k], where=[(i > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -618,20 +618,20 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(f1=[$3]) LogicalFilter(condition=[=($cor1.a, $0)]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })], variablesSet=[[$cor1]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -645,9 +645,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, SUBSTRING($2, 1, 5), { LogicalProject(d=[$0], EXPR$1=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -655,10 +655,10 @@ LogicalProject(d=[$0], EXPR$1=[SUBSTRING($2, 1, 5)]) Calc(select=[a, b, c]) +- HashJoin(joinType=[LeftSemiJoin], where=[((a = d) AND ($f3 = EXPR$1) AND (b = e))], select=[a, b, c, $f3], isBroadcast=[true], build=[right]) :- Calc(select=[a, b, c, SUBSTRING(c, 1, 5) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, SUBSTRING(f, 1, 5) AS EXPR$1, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -672,18 +672,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> e))], select=[a, b, c], isBroadcast=[true], build=[right]) -:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +:- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -697,19 +697,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($cor0.b, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -724,15 +724,15 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(rk=[$2]) LogicalFilter(condition=[<>($cor0.a, $0)]) LogicalProject(d=[$0], e=[$1], rk=[RANK() OVER (PARTITION BY $0 ORDER BY $1 NULLS FIRST)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> d))], select=[a, b, c], isBroadcast=[true], build=[right]) -:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +:- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[w0$o0 AS rk, d]) +- OverAggregate(partitionBy=[d], orderBy=[e ASC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[d, e, w0$o0]) @@ -740,7 +740,7 @@ HashJoin(joinType=[LeftSemiJoin], where=[((b = rk) AND (a <> d))], select=[a, b, +- Sort(orderBy=[d ASC, e ASC]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -756,15 +756,15 @@ LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) LogicalProject(d=[$0]) LogicalFilter(condition=[IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), { LogicalProject(i=[$0]) LogicalFilter(condition=[=($2, $cor0.c)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -773,9 +773,9 @@ Calc(select=[a]) +- MultipleInput(readOrder=[0,1,0], members=[\nHashJoin(joinType=[LeftSemiJoin], where=[((EXPR$0 = i) AND (k = c))], select=[a, c, EXPR$0], isBroadcast=[true], build=[right])\n:- NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[a, c, EXPR$0], build=[right], singleRowJoin=[true])\n: :- [#2] Calc(select=[a, c])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[i, k]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) :- Calc(select=[a, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- HashAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0]) +- Exchange(distribution=[single]) @@ -783,7 +783,7 @@ Calc(select=[a]) +- Calc(select=[d]) +- HashJoin(joinType=[InnerJoin], where=[(e = j)], select=[d, e, j], isBroadcast=[true], build=[right]) :- Calc(select=[d, e]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[broadcast]) +- HashAggregate(isMerge=[true], groupBy=[j], select=[j]) +- Exchange(distribution=[hash[j]]) @@ -803,18 +803,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -828,19 +828,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 1), IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, e], where=[(d > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -854,19 +854,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=(CAST($cor0.b):INTEGER, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -882,9 +882,9 @@ LogicalProject(EXPR$0=[$1]) LogicalAggregate(group=[{0}], EXPR$0=[MAX($1)]) LogicalProject(f=[$2], e=[$1]) LogicalFilter(condition=[<($0, 3)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -892,14 +892,14 @@ LogicalProject(EXPR$0=[$1]) Calc(select=[a]) +- HashJoin(joinType=[LeftSemiJoin], where=[(b = EXPR$0)], select=[a, b], isBroadcast=[true], build=[right]) :- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[EXPR$0]) +- HashAggregate(isMerge=[true], groupBy=[f], select=[f, Final_MAX(max$0) AS EXPR$0]) +- Exchange(distribution=[hash[f]]) +- LocalHashAggregate(groupBy=[f], select=[f, Partial_MAX(e) AS max$0]) +- Calc(select=[f, e], where=[(d < 3)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -913,21 +913,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, $1, { LogicalAggregate(group=[{0}], EXPR$1=[COUNT()]) LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -940,19 +940,19 @@ HashJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (b = EXPR$1))], select=[a, LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), NOT(LIKE($2, _UTF-16LE'abc')), IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10) AND NOT(LIKE(c, 'abc')))]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -965,18 +965,18 @@ HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], isBroadcast LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IS TRUE(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -991,12 +991,12 @@ LogicalProject(a=[$0]) LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) LogicalProject(e=[$1]) LogicalFilter(condition=[>($0, 0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1005,15 +1005,15 @@ Calc(select=[a]) +- MultipleInput(readOrder=[0,1,0], members=[\nHashJoin(joinType=[LeftSemiJoin], where=[(EXPR$0 = j)], select=[a, EXPR$0], isBroadcast=[true], build=[right])\n:- NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[a, EXPR$0], build=[right], singleRowJoin=[true])\n: :- [#2] Calc(select=[a])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[j]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) :- Calc(select=[a]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- HashAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MAX(e) AS max$0]) +- Calc(select=[e], where=[(d > 0)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1027,15 +1027,15 @@ LogicalProject(b=[$1]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($1, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), >($1, $SCALAR_QUERY({ LogicalProject(EXPR$0=[*(0.5:DECIMAL(2, 1), $0)]) LogicalAggregate(group=[{}], agg#0=[SUM($0)]) LogicalProject(j=[$1]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1049,12 +1049,12 @@ Calc(select=[b]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_SUM(j) AS sum$0]) : +- Calc(select=[j], where=[(i < 100)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) :- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d], where=[(e > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1067,11 +1067,11 @@ Calc(select=[b]) LogicalProject(s=[$1]) +- LogicalFilter(condition=[AND(>($2, 2), IN($3, { LogicalProject(e=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) +- LogicalAggregate(group=[{0}], s=[SUM($1)], agg#1=[COUNT()], agg#2=[MAX($0)]) +- LogicalProject(b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1080,11 +1080,11 @@ Calc(select=[s]) +- MultipleInput(readOrder=[0,1], members=[\nHashJoin(joinType=[LeftSemiJoin], where=[($f3 = e)], select=[s, $f3], isBroadcast=[true], build=[right])\n:- Calc(select=[s, $f3], where=[($f2 > 2)])\n: +- HashAggregate(isMerge=[true], groupBy=[b], select=[b, Final_SUM(sum$0) AS s, Final_COUNT(count1$1) AS $f2, Final_MAX(max$2) AS $f3])\n: +- [#2] Exchange(distribution=[hash[b]])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[e]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[b]]) +- LocalHashAggregate(groupBy=[b], select=[b, Partial_SUM(a) AS sum$0, Partial_COUNT(*) AS count1$1, Partial_MAX(b) AS max$2]) +- Calc(select=[b, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) ]]> @@ -1098,20 +1098,20 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($2, { LogicalProject(f1=[$3]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1127,9 +1127,9 @@ SELECT * FROM l WHERE LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN(+($0, 10), SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[+($0, 100)], EXPR$1=[SUBSTRING($2, 1, 5)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1137,10 +1137,10 @@ LogicalProject(EXPR$0=[+($0, 100)], EXPR$1=[SUBSTRING($2, 1, 5)]) Calc(select=[a, b, c]) +- HashJoin(joinType=[LeftSemiJoin], where=[(($f3 = EXPR$0) AND ($f4 = EXPR$1))], select=[a, b, c, $f3, $f4], isBroadcast=[true], build=[right]) :- Calc(select=[a, b, c, (a + 10) AS $f3, SUBSTRING(c, 1, 5) AS $f4]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[(d + 100) AS EXPR$0, SUBSTRING(f, 1, 5) AS EXPR$1]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1156,15 +1156,15 @@ SELECT * FROM l WHERE (a, b) IN LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, $1, { LogicalProject(EXPR$0=[MAX($0) OVER ()], EXPR$1=[MIN($1) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1186,18 +1186,18 @@ HashJoin(joinType=[LeftSemiJoin], where=[((a = $0) AND (b = $1))], select=[a, b, LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1211,19 +1211,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), >($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d], where=[(e < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1236,9 +1236,9 @@ HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], isBroadcast LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN(+($0, 1), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1246,10 +1246,10 @@ LogicalProject(d=[$0]) Calc(select=[a, b, c]) +- HashJoin(joinType=[LeftSemiJoin], where=[($f3 = d)], select=[a, b, c, $f3], isBroadcast=[true], build=[right]) :- Calc(select=[a, b, c, (a + 1) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1267,12 +1267,12 @@ LogicalProject(a=[$0]) LogicalUnion(all=[false]) LogicalProject(e=[$1]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1280,16 +1280,16 @@ LogicalUnion(all=[false]) Calc(select=[a]) +- HashJoin(joinType=[LeftSemiJoin], where=[(b = e)], select=[a, b], isBroadcast=[true], build=[right]) :- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- HashAggregate(isMerge=[true], groupBy=[e], select=[e]) +- Exchange(distribution=[hash[e]]) +- LocalHashAggregate(groupBy=[e], select=[e]) +- Union(all=[true], union=[e]) :- Calc(select=[e], where=[(d > 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Calc(select=[CAST(i AS BIGINT) AS i], where=[(i < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1302,28 +1302,28 @@ Calc(select=[a]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[AND(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) +LogicalTableScan(table=[[default_catalog, default_database, t]]) }), =($cor0.a, $0))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1336,26 +1336,26 @@ HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], isBroadcast LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), EXISTS({ LogicalFilter(condition=[>($0, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), >=($1, 1), EXISTS({ LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> = 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) ++- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1379,29 +1379,29 @@ MultipleInput(readOrder=[0,0,0,1], members=[\nNestedLoopJoin(joinType=[LeftSemiJ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), NOT(EXISTS({ LogicalFilter(condition=[AND(=($cor0.b, $1), >(CAST($2):BIGINT, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), >=(CAST($2):BIGINT, 1), NOT(EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> 50)]) : +- Reused(reference_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a, b, c], where=[(CAST(c AS BIGINT) >= 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) ++- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1415,25 +1415,25 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) LogicalFilter(condition=[AND(=($cor0.a, $0), <>($2, _UTF-16LE'test'))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 'test')]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) -:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) +:- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, f]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1449,24 +1449,24 @@ LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), IN($2, { LogicalProject(k=[$2]) LogicalFilter(condition=[=($cor1.e, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor1]]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1479,25 +1479,25 @@ HashJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (b = e))], select=[a, b, c LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) LogicalFilter(condition=[=($2, $cor0.c)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1510,24 +1510,24 @@ MultipleInput(readOrder=[0,1,0], members=[\nHashJoin(joinType=[LeftSemiJoin], wh LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1542,25 +1542,25 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(d=[$0]) LogicalFilter(condition=[IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1573,26 +1573,26 @@ HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], isBroadcast LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), NOT(EXISTS({ LogicalFilter(condition=[>($0, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), >=($1, 1), NOT(EXISTS({ LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> = 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) ++- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1616,18 +1616,18 @@ MultipleInput(readOrder=[0,0,0,1], members=[\nNestedLoopJoin(joinType=[LeftAntiJ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1639,9 +1639,9 @@ HashJoin(joinType=[LeftAntiJoin], where=[(a = d)], select=[a, b, c], isBroadcast ($1, 10), LIKE($2, _UTF-16LE'abc'), NOT(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) +LogicalTableScan(table=[[default_catalog, default_database, r]]) }))))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1649,14 +1649,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl Calc(select=[(a + 10) AS EXPR$0, c]) +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, c], build=[right], singleRowJoin=[true]) :- Calc(select=[a, c], where=[((b <= 10) AND NOT(LIKE(c, 'abc')))]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[], where=[m IS NOT NULL]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1668,22 +1668,22 @@ Calc(select=[(a + 10) AS EXPR$0, c]) @@ -1695,23 +1695,23 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c], build=[ri ($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[m IS NOT NULL AS $f0]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1726,21 +1726,21 @@ LogicalProject(a=[$0], b=[$1]) LogicalFilter(condition=[=($cor0.a, $0)]) LogicalAggregate(group=[{0}]) LogicalProject(c=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, rightT]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, leftT]]) ]]> @@ -1753,25 +1753,25 @@ HashJoin(joinType=[LeftAntiJoin], where=[(a = c)], select=[a, b], isBroadcast=[t LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })), NOT(IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1785,9 +1785,9 @@ LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) +- LogicalFilter(condition=[NOT(IN(SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=(+($cor0.b, 1), $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1795,10 +1795,10 @@ LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)]) Calc(select=[(a + 10) AS EXPR$0, c]) +- HashJoin(joinType=[LeftAntiJoin], where=[(($f3 IS NULL OR EXPR$0 IS NULL OR ($f3 = EXPR$0)) AND ($f4 = e))], select=[a, c, $f3, $f4], isBroadcast=[true], build=[right]) :- Calc(select=[a, c, SUBSTRING(c, 1, 5) AS $f3, (b + 1) AS $f4]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[SUBSTRING(f, 1, 5) AS EXPR$0, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1811,32 +1811,32 @@ Calc(select=[(a + 10) AS EXPR$0, c]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[NOT(IN(CASE(NOT(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), 1, NOT(IN($0, { LogicalProject(EXPR$0=[CAST($1):INTEGER]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), 2, 3), { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> = c0) AND a IS NOT NULL)), 1, ((c1 = 0) OR (i IS NULL AND (ck0 >= c1) AND a IS NOT NULL)), 2, 3) AS $f3])\n: +- HashJoin(joinType=[LeftOuterJoin], where=[(a = EXPR$0)], select=[a, b, c, c0, ck, i0, c1, ck0, EXPR$0, i], isBroadcast=[true], build=[right])\n: :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true])\n: : :- Calc(select=[a, b, c, c0, ck, i0])\n: : : +- HashJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, c0, ck, i, i0], isBroadcast=[true], build=[right])\n: : : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true])\n: : : : :- [#5] LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])\n: : : : +- [#6] Exchange(distribution=[broadcast])\n: : : +- [#4] Exchange(distribution=[broadcast])\n: : +- [#3] Exchange(distribution=[broadcast])\n: +- [#2] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) ++- MultipleInput(readOrder=[0,0,0,0,1,0], members=[\nHashJoin(joinType=[LeftAntiJoin], where=[((d IS NULL OR ($f3 = d)) AND (c = f))], select=[b, c, $f3], isBroadcast=[true], build=[right])\n:- Calc(select=[b, c, CASE(((c0 = 0) OR (i0 IS NULL AND (ck >= c0) AND a IS NOT NULL)), 1, ((c1 = 0) OR (i IS NULL AND (ck0 >= c1) AND a IS NOT NULL)), 2, 3) AS $f3])\n: +- HashJoin(joinType=[LeftOuterJoin], where=[(a = EXPR$0)], select=[a, b, c, c0, ck, i0, c1, ck0, EXPR$0, i], isBroadcast=[true], build=[right])\n: :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true])\n: : :- Calc(select=[a, b, c, c0, ck, i0])\n: : : +- HashJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, c0, ck, i, i0], isBroadcast=[true], build=[right])\n: : : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true])\n: : : : :- [#5] TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c])\n: : : : +- [#6] Exchange(distribution=[broadcast])\n: : : +- [#4] Exchange(distribution=[broadcast])\n: : +- [#3] Exchange(distribution=[broadcast])\n: +- [#2] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[d, f]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[EXPR$0, true AS i]) : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) : +- Exchange(distribution=[hash[EXPR$0]]) : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) : +- Calc(select=[CAST(j AS INTEGER) AS EXPR$0])(reuse_id=[1]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[2]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[2]) :- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : +- Exchange(distribution=[single]) @@ -1849,7 +1849,7 @@ Calc(select=[b]) : +- LocalHashAggregate(groupBy=[i], select=[i]) : +- Calc(select=[i])(reuse_id=[3]) : +- Reused(reference_id=[2]) - :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + :- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) +- Exchange(distribution=[single]) @@ -1868,18 +1868,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, $2, { LogicalProject(d=[$0], f=[$2]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1893,18 +1893,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1918,19 +1918,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 1), NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, e], where=[(d > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml index 680cef7d722d9..c796e2645afa0 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml @@ -24,16 +24,16 @@ limitations under the License. @@ -45,17 +45,17 @@ NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[d, e, f, g, h, a, b, @@ -67,17 +67,17 @@ NestedLoopJoin(joinType=[FullOuterJoin], where=[false], select=[d, e, f, g, h, a @@ -89,8 +89,8 @@ NestedLoopJoin(joinType=[FullOuterJoin], where=[true], select=[d, e, f, g, h, a, @@ -99,10 +99,10 @@ Calc(select=[c, g]) +- NestedLoopJoin(joinType=[FullOuterJoin], where=[((a = d) AND (d < 2))], select=[d, g, a, c], build=[left]) :- Exchange(distribution=[single]) : +- Calc(select=[d, g]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[single]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -114,17 +114,17 @@ Calc(select=[c, g]) @@ -136,8 +136,8 @@ NestedLoopJoin(joinType=[FullOuterJoin], where=[((a = d) AND (d < 2) AND (b < h) @@ -146,10 +146,10 @@ Calc(select=[c, g]) +- NestedLoopJoin(joinType=[FullOuterJoin], where=[(b = e)], select=[b, c, e, g], build=[left]) :- Exchange(distribution=[single]) : +- Calc(select=[b, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[single]) +- Calc(select=[e, g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -161,17 +161,17 @@ Calc(select=[c, g]) ($5, $0)], joinType=[full]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> d)], select=[d, e, f, g, h, a, b, c], build=[right]) :- Exchange(distribution=[single]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[single]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -186,9 +186,9 @@ SELECT * FROM (SELECT * FROM MyTable1) FULL JOIN (SELECT * FROM MyTable3) USING LogicalProject(a=[COALESCE($0, $3)], b=[$1], c=[$2], b0=[$4], c0=[$5]) +- LogicalJoin(condition=[=($0, $3)], joinType=[full]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3]]) ]]> @@ -196,9 +196,9 @@ LogicalProject(a=[COALESCE($0, $3)], b=[$1], c=[$2], b0=[$4], c0=[$5]) Calc(select=[COALESCE(a, a0) AS a, b, c, b0, c0]) +- NestedLoopJoin(joinType=[FullOuterJoin], where=[(a = a0)], select=[a, b, c, a0, b0, c0], build=[left]) :- Exchange(distribution=[single]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[single]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable3]], fields=[a, b, c]) ]]> @@ -210,8 +210,8 @@ Calc(select=[COALESCE(a, a0) AS a, b, c, b0, c0]) @@ -220,9 +220,9 @@ Calc(select=[c, g]) +- NestedLoopJoin(joinType=[InnerJoin], where=[(a = d)], select=[d, g, a, c], build=[left]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[d, g], where=[(d < 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Calc(select=[a, c], where=[(a < 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -234,18 +234,18 @@ Calc(select=[c, g]) @@ -258,8 +258,8 @@ NestedLoopJoin(joinType=[InnerJoin], where=[((a = d) AND (b < h))], select=[d, e LogicalProject(c=[$2], g=[$6]) +- LogicalFilter(condition=[=($0, $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -268,9 +268,9 @@ Calc(select=[c, g]) +- NestedLoopJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, c, d, g], build=[left]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Calc(select=[d, g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -283,8 +283,8 @@ Calc(select=[c, g]) LogicalProject(c=[$7], g=[$3]) +- LogicalFilter(condition=[AND(=($5, $0), <($0, 2))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -293,9 +293,9 @@ Calc(select=[c, g]) +- NestedLoopJoin(joinType=[InnerJoin], where=[(a = d)], select=[d, g, a, c], build=[left]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[d, g], where=[(d < 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Calc(select=[a, c], where=[(a < 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -316,10 +316,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -331,11 +331,11 @@ Calc(select=[a, b, CAST(2 AS INTEGER) AS d, e]) : +- Exchange(distribution=[hash[a]]) : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) : +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -348,8 +348,8 @@ Calc(select=[a, b, CAST(2 AS INTEGER) AS d, e]) LogicalProject(c=[$2], g=[$6]) +- LogicalFilter(condition=[AND(=($1, $4), =($0, $3))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -357,9 +357,9 @@ LogicalProject(c=[$2], g=[$6]) Calc(select=[c, g]) +- NestedLoopJoin(joinType=[InnerJoin], where=[((b = e) AND (a = d))], select=[a, b, c, d, e, g], build=[left]) :- Exchange(distribution=[broadcast]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Calc(select=[d, e, g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -379,10 +379,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($2, 2), =($1, 1))], joinType=[inner]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -395,11 +395,11 @@ Calc(select=[a, 1 AS b, CAST(2 AS INTEGER) AS d, 1 AS e]) : +- Exchange(distribution=[hash[a]]) : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) : +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -411,8 +411,8 @@ Calc(select=[a, 1 AS b, CAST(2 AS INTEGER) AS d, 1 AS e]) @@ -421,8 +421,8 @@ Calc(select=[c, g]) +- NestedLoopJoin(joinType=[InnerJoin], where=[((a = d) AND (b = e))], select=[d, e, g, a, b, c], build=[left]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[d, e, g]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -435,8 +435,8 @@ Calc(select=[c, g]) LogicalProject(a=[$0], d=[$3]) +- LogicalFilter(condition=[=(+($0, 1), $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -444,10 +444,10 @@ LogicalProject(a=[$0], d=[$3]) Calc(select=[a, d]) +- NestedLoopJoin(joinType=[InnerJoin], where=[($f3 = d)], select=[a, $f3, d], build=[right]) :- Calc(select=[a, (a + 1) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -459,8 +459,8 @@ Calc(select=[a, d]) @@ -468,9 +468,9 @@ LogicalProject(a=[$0], d=[$3]) NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, d], build=[left]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -512,10 +512,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[true], joinType=[left]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -527,11 +527,11 @@ Calc(select=[CAST(2 AS INTEGER) AS a, b, d, CAST(e AS BIGINT) AS e]) : +- Exchange(distribution=[hash[a]]) : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) : +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -551,10 +551,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($0, 2), =($3, 1))], joinType=[left]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -566,11 +566,11 @@ MultipleInput(readOrder=[0,1], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin : +- Exchange(distribution=[hash[d]]) : +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) : +- Calc(select=[d, e]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -582,16 +582,16 @@ MultipleInput(readOrder=[0,1], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin ($5, $0)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> d)], select=[d, e, f, g, h, a, b, c], build=[right]) -:- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) +:- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[broadcast]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -603,14 +603,14 @@ NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a <> d)], select=[d, e, f, g, h @@ -622,16 +622,16 @@ Calc(select=[d, e, f, g, h, null:INTEGER AS a, null:BIGINT AS b, null:VARCHAR(21 @@ -643,8 +643,8 @@ NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[d, e, f, g, h, a, @@ -652,10 +652,10 @@ LogicalProject(c=[$7], g=[$3]) Calc(select=[c, g]) +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[((a = d) AND (d < 2))], select=[d, g, a, c], build=[right]) :- Calc(select=[d, g]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[broadcast]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -668,8 +668,8 @@ Calc(select=[c, g]) LogicalProject(d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($3, 10), <($0, 12))]) +- LogicalJoin(condition=[=($0, $3)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -678,9 +678,9 @@ Calc(select=[CAST(10 AS INTEGER) AS d, e, f]) +- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, e, f], build=[left]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a], where=[(a = 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Calc(select=[e, f], where=[(d = 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -692,16 +692,16 @@ Calc(select=[CAST(10 AS INTEGER) AS d, e, f]) @@ -713,8 +713,8 @@ NestedLoopJoin(joinType=[LeftOuterJoin], where=[((a = d) AND (d < 2) AND (b < h) @@ -722,10 +722,10 @@ LogicalProject(c=[$2], g=[$6]) Calc(select=[c, g]) +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(b = e)], select=[b, c, e, g], build=[right]) :- Calc(select=[b, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[e, g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -738,8 +738,8 @@ Calc(select=[c, g]) LogicalProject(d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(IS NULL($3), <($0, 12))]) +- LogicalJoin(condition=[=($0, $3)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -747,10 +747,10 @@ LogicalProject(d=[$3], e=[$4], f=[$5]) Calc(select=[null:INTEGER AS d, e, f], where=[d IS NULL]) +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = d)], select=[a, d, e, f], build=[right]) :- Calc(select=[a], where=[(a < 12)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, e, f], where=[(d < 12)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -763,8 +763,8 @@ Calc(select=[null:INTEGER AS d, e, f], where=[d IS NULL]) LogicalProject(d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(<($3, 10), <($0, 12))]) +- LogicalJoin(condition=[=($0, $3)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -773,9 +773,9 @@ Calc(select=[d, e, f]) +- NestedLoopJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d, e, f], build=[left]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a], where=[(a < 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Calc(select=[d, e, f], where=[(d < 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -788,8 +788,8 @@ Calc(select=[d, e, f]) LogicalProject(d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[=($3, null)]) +- LogicalJoin(condition=[=($0, $3)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -815,10 +815,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[true], joinType=[right]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -830,11 +830,11 @@ Calc(select=[a, CAST(b AS BIGINT) AS b, CAST(2 AS INTEGER) AS d, e]) : +- Exchange(distribution=[hash[a]]) : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) : +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -846,8 +846,8 @@ Calc(select=[a, CAST(b AS BIGINT) AS b, CAST(2 AS INTEGER) AS d, e]) @@ -856,9 +856,9 @@ Calc(select=[c, g]) +- NestedLoopJoin(joinType=[RightOuterJoin], where=[(b = e)], select=[b, c, e, g], build=[left]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[b, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Calc(select=[e, g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -878,10 +878,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($2, 2), =($1, 1))], joinType=[right]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -893,11 +893,11 @@ MultipleInput(readOrder=[0,1], members=[\nNestedLoopJoin(joinType=[RightOuterJoi : +- Exchange(distribution=[hash[a]]) : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) : +- Calc(select=[a, b]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -909,14 +909,14 @@ MultipleInput(readOrder=[0,1], members=[\nNestedLoopJoin(joinType=[RightOuterJoi @@ -928,16 +928,16 @@ Calc(select=[null:INTEGER AS d, null:BIGINT AS e, null:INTEGER AS f, null:VARCHA @@ -949,8 +949,8 @@ NestedLoopJoin(joinType=[RightOuterJoin], where=[true], select=[d, e, f, g, h, a @@ -959,9 +959,9 @@ Calc(select=[c, g]) +- NestedLoopJoin(joinType=[RightOuterJoin], where=[(a = d)], select=[d, g, a, c], build=[left]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[d, g], where=[(d < 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -973,16 +973,16 @@ Calc(select=[c, g]) ($5, $0)], joinType=[right]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> d)], select=[d, e, f, g, h, a, b, c], build=[left]) :- Exchange(distribution=[broadcast]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -994,8 +994,8 @@ NestedLoopJoin(joinType=[RightOuterJoin], where=[(a <> d)], select=[d, e, f, g, @@ -1003,8 +1003,8 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7]) NestedLoopJoin(joinType=[RightOuterJoin], where=[((a = d) AND (b < h))], select=[d, e, f, g, h, a, b, c], build=[left]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[d, e, f, g, h], where=[(d < 2)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -1023,16 +1023,16 @@ LogicalProject(k=[$0], v=[$1], k0=[$2], v0=[$3]) +- LogicalJoin(condition=[AND(=($0, $2), >($2, 10))], joinType=[left]) :- LogicalProject(k=[$0], v=[$1]) : +- LogicalFilter(condition=[=($0, 0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, src]]) +- LogicalProject(k=[$0], v=[$1]) +- LogicalFilter(condition=[=($0, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, src]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml index c448697a17c2a..094c0b1f4f2c3 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml @@ -24,28 +24,28 @@ limitations under the License. @@ -60,22 +60,22 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$0=[MAX($4)]) LogicalProject(d=[$0], $f1=[true], f=[$2], $f3=[1], e=[$1]) LogicalFilter(condition=[AND(=($cor0.b, $1), <($0, 100), =($cor0.c, $2))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -89,20 +89,20 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=($cor1.a, $0)]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })], variablesSet=[[$cor1]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -115,18 +115,18 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[<($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -139,20 +139,20 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[(a < d)], select=[a, b, c], build LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[<($cor0.a, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -167,22 +167,22 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalAggregate(group=[{0}]) LogicalProject(e=[$1]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -195,18 +195,18 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[(c = f)], select=[a, b, c], build LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -218,9 +218,9 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build ($1, 10), NOT(OR(LIKE($2, _UTF-16LE'abc'), NOT(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) +LogicalTableScan(table=[[default_catalog, default_database, r]]) })))))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -228,14 +228,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl Calc(select=[(a + 10) AS EXPR$0, c]) +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, c], build=[right], singleRowJoin=[true]) :- Calc(select=[a, c], where=[((b > 10) AND NOT(LIKE(c, 'abc')))]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[], where=[m IS NOT NULL]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -248,19 +248,19 @@ Calc(select=[(a + 10) AS EXPR$0, c]) LogicalProject(c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalJoin(condition=[=($1, $3)], joinType=[full]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -286,19 +286,19 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[c], build=[right], LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) +LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -327,22 +327,22 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], build=[r @@ -354,23 +354,23 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], build=[r ($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[], where=[m IS NOT NULL]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -385,19 +385,19 @@ LogicalProject(a=[$0]) LogicalUnion(all=[false]) LogicalProject(e=[$1], f=[$2]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1], k=[$2]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Calc(select=[j, k], where=[(i < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -424,25 +424,25 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a], build=[right], LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -455,24 +455,24 @@ MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftSemiJoi LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -485,32 +485,32 @@ MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftSemiJoi LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), NOT(IN($1, { LogicalProject(i=[$0]) LogicalFilter(condition=[AND(=($cor0.c, $2), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), EXISTS({ LogicalAggregate(group=[{0}], EXPR$0=[COUNT($0)]) LogicalProject(l=[$0]) LogicalFilter(condition=[LIKE($2, _UTF-16LE'Test')]) - LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t2]]) }), NOT(EXISTS({ LogicalFilter(condition=[<>($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> e)], select=[a, b, c], build=[right])\n:- NestedLoopJoin(joinType=[LeftAntiJoin], where=[((b IS NULL OR i IS NULL OR (b = i)) AND (c = k))], select=[a, b, c], build=[right])\n: :- NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], build=[right], singleRowJoin=[true])\n: : :- NestedLoopJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[right])\n: : : :- [#4] LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])\n: : : +- [#5] Exchange(distribution=[broadcast])\n: : +- [#3] Exchange(distribution=[broadcast])\n: +- [#2] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) +MultipleInput(readOrder=[0,0,0,1,0], members=[\nNestedLoopJoin(joinType=[LeftAntiJoin], where=[(b <> e)], select=[a, b, c], build=[right])\n:- NestedLoopJoin(joinType=[LeftAntiJoin], where=[((b IS NULL OR i IS NULL OR (b = i)) AND (c = k))], select=[a, b, c], build=[right])\n: :- NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], build=[right], singleRowJoin=[true])\n: : :- NestedLoopJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[right])\n: : : :- [#4] TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c])\n: : : +- [#5] Exchange(distribution=[broadcast])\n: : +- [#3] Exchange(distribution=[broadcast])\n: +- [#2] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[e]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])(reuse_id=[1]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f])(reuse_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[i, k], where=[(i > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[true AS $f0], where=[m IS NOT NULL]) : +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) @@ -521,8 +521,8 @@ MultipleInput(readOrder=[0,0,0,1,0], members=[\nNestedLoopJoin(joinType=[LeftAnt : +- Exchange(distribution=[hash[l]]) : +- LocalHashAggregate(groupBy=[l], select=[l]) : +- Calc(select=[l], where=[LIKE(n, 'Test')]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]], fields=[l, m, n]) -:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, t2]], fields=[l, m, n]) +:- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) +- Reused(reference_id=[1]) @@ -541,22 +541,22 @@ LogicalProject(EXPR$0=[$1]) LogicalAggregate(group=[{0}], EXPR$0=[MAX($1)]) LogicalProject(f=[$2], e=[$1]) LogicalFilter(condition=[AND(=($cor0.c, $2), <($0, 3))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -572,21 +572,21 @@ LogicalProject(EXPR$0=[$4], d=[$0]) LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$0=[COUNT()]) LogicalProject(d=[$0], $f1=[true], e=[$1], $f3=[1]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -600,9 +600,9 @@ LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) +- LogicalFilter(condition=[IN(SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=(+($cor0.b, 1), $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -610,10 +610,10 @@ LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)]) Calc(select=[(a + 10) AS EXPR$0, c]) +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[(($f3 = EXPR$0) AND ($f4 = e))], select=[a, c, $f3, $f4], build=[right]) :- Calc(select=[a, c, SUBSTRING(c, 1, 5) AS $f3, (b + 1) AS $f4]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[SUBSTRING(f, 1, 5) AS EXPR$0, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -627,19 +627,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), NOT(LIKE($2, _UTF-16LE'abc')), IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10) AND NOT(LIKE(c, 'abc')))]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -652,30 +652,30 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (b = e))], select=[a LogicalProject(b=[$1]) +- LogicalFilter(condition=[IN(CASE(IN($0, { LogicalProject(EXPR$0=[1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), 1, 2), { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 0) AND i IS NOT NULL AND a IS NOT NULL), 1, 2) AS $f3])\n: +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = 1)], select=[a, b, c, c0, i], build=[right])\n: :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0], build=[right], singleRowJoin=[true])\n: : :- [#3] LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])\n: : +- [#4] Exchange(distribution=[broadcast])\n: +- [#2] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) ++- MultipleInput(readOrder=[0,0,1,0], members=[\nNestedLoopJoin(joinType=[LeftSemiJoin], where=[(($f3 = d) AND (c = f))], select=[b, c, $f3], build=[right])\n:- Calc(select=[b, c, CASE(((c0 <> 0) AND i IS NOT NULL AND a IS NOT NULL), 1, 2) AS $f3])\n: +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = 1)], select=[a, b, c, c0, i], build=[right])\n: :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0], build=[right], singleRowJoin=[true])\n: : :- [#3] TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c])\n: : +- [#4] Exchange(distribution=[broadcast])\n: +- [#2] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[d, f]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[true AS i]) : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) : +- Exchange(distribution=[hash[EXPR$0]]) : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) : +- Calc(select=[1 AS EXPR$0]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) - :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) + :- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[single]) @@ -696,27 +696,27 @@ LogicalProject(e=[$0]) LogicalJoin(condition=[=($0, $1)], joinType=[inner]) LogicalProject(e=[$1]) LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1]) LogicalFilter(condition=[AND(=($cor0.c, $2), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -732,25 +732,25 @@ LogicalProject(j=[$3]) LogicalJoin(condition=[=($1, $4)], joinType=[left]) LogicalProject(e=[$1], f=[$2]) LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -764,12 +764,12 @@ LogicalProject(c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalJoin(condition=[=($0, $3)], joinType=[right]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0]) LogicalFilter(condition=[AND(=($cor0.c, $2), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -777,15 +777,15 @@ LogicalProject(d=[$0]) Calc(select=[c]) +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (c = k))], select=[a, c], build=[right]) :- Calc(select=[a, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, k]) +- NestedLoopJoin(joinType=[RightOuterJoin], where=[(d = i)], select=[d, i, k], build=[left]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[d]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Calc(select=[i, k], where=[(i > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -800,20 +800,20 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(f1=[$3]) LogicalFilter(condition=[=($cor1.a, $0)]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })], variablesSet=[[$cor1]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -827,9 +827,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, SUBSTRING($2, 1, 5), { LogicalProject(d=[$0], EXPR$1=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -837,10 +837,10 @@ LogicalProject(d=[$0], EXPR$1=[SUBSTRING($2, 1, 5)]) Calc(select=[a, b, c]) +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[((a = d) AND ($f3 = EXPR$1) AND (b = e))], select=[a, b, c, $f3], build=[right]) :- Calc(select=[a, b, c, SUBSTRING(c, 1, 5) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, SUBSTRING(f, 1, 5) AS EXPR$1, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -854,18 +854,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> e))], select=[a, b, c], build=[right]) -:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +:- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -879,19 +879,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($cor0.b, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -906,15 +906,15 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(rk=[$2]) LogicalFilter(condition=[<>($cor0.a, $0)]) LogicalProject(d=[$0], e=[$1], rk=[RANK() OVER (PARTITION BY $0 ORDER BY $1 NULLS FIRST)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> d))], select=[a, b, c], build=[right]) -:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +:- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[w0$o0 AS rk, d]) +- OverAggregate(partitionBy=[d], orderBy=[e ASC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[d, e, w0$o0]) @@ -922,7 +922,7 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[((b = rk) AND (a <> d))], select= +- Sort(orderBy=[d ASC, e ASC]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -938,15 +938,15 @@ LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) LogicalProject(d=[$0]) LogicalFilter(condition=[IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), { LogicalProject(i=[$0]) LogicalFilter(condition=[=($2, $cor0.c)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -955,9 +955,9 @@ Calc(select=[a]) +- MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftSemiJoin], where=[((EXPR$0 = i) AND (k = c))], select=[a, c, EXPR$0], build=[right])\n:- NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[a, c, EXPR$0], build=[right], singleRowJoin=[true])\n: :- [#2] Calc(select=[a, c])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[i, k]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) :- Calc(select=[a, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- HashAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0]) +- Exchange(distribution=[single]) @@ -965,7 +965,7 @@ Calc(select=[a]) +- Calc(select=[d]) +- NestedLoopJoin(joinType=[InnerJoin], where=[(e = j)], select=[d, e, j], build=[right]) :- Calc(select=[d, e]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[broadcast]) +- HashAggregate(isMerge=[true], groupBy=[j], select=[j]) +- Exchange(distribution=[hash[j]]) @@ -985,18 +985,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1010,19 +1010,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 1), IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, e], where=[(d > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1036,19 +1036,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=(CAST($cor0.b):INTEGER, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1064,9 +1064,9 @@ LogicalProject(EXPR$0=[$1]) LogicalAggregate(group=[{0}], EXPR$0=[MAX($1)]) LogicalProject(f=[$2], e=[$1]) LogicalFilter(condition=[<($0, 3)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1074,14 +1074,14 @@ LogicalProject(EXPR$0=[$1]) Calc(select=[a]) +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[(b = EXPR$0)], select=[a, b], build=[right]) :- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[EXPR$0]) +- HashAggregate(isMerge=[true], groupBy=[f], select=[f, Final_MAX(max$0) AS EXPR$0]) +- Exchange(distribution=[hash[f]]) +- LocalHashAggregate(groupBy=[f], select=[f, Partial_MAX(e) AS max$0]) +- Calc(select=[f, e], where=[(d < 3)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1095,21 +1095,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, $1, { LogicalAggregate(group=[{0}], EXPR$1=[COUNT()]) LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1122,19 +1122,19 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (b = EXPR$1))], sele LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), NOT(LIKE($2, _UTF-16LE'abc')), IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10) AND NOT(LIKE(c, 'abc')))]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1147,18 +1147,18 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IS TRUE(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1174,12 +1174,12 @@ SELECT b FROM l WHERE (CASE WHEN a IN LogicalProject(b=[$1]) +- LogicalFilter(condition=[IN(CASE(IN($0, { LogicalProject(EXPR$0=[1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), 1, 2), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1188,16 +1188,16 @@ Calc(select=[b]) +- MultipleInput(readOrder=[0,0,1,0], members=[\nNestedLoopJoin(joinType=[LeftSemiJoin], where=[($f3 = d)], select=[b, $f3], build=[right])\n:- Calc(select=[b, CASE(((c <> 0) AND i IS NOT NULL AND a IS NOT NULL), 1, 2) AS $f3])\n: +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = 1)], select=[a, b, c, i], build=[right])\n: :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c], build=[right], singleRowJoin=[true])\n: : :- [#3] Calc(select=[a, b])\n: : +- [#4] Exchange(distribution=[broadcast])\n: +- [#2] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[d]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[true AS i]) : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) : +- Exchange(distribution=[hash[EXPR$0]]) : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) : +- Calc(select=[1 AS EXPR$0]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) :- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[single]) @@ -1217,12 +1217,12 @@ LogicalProject(a=[$0]) LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) LogicalProject(e=[$1]) LogicalFilter(condition=[>($0, 0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1231,15 +1231,15 @@ Calc(select=[a]) +- MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftSemiJoin], where=[(EXPR$0 = j)], select=[a, EXPR$0], build=[right])\n:- NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[a, EXPR$0], build=[right], singleRowJoin=[true])\n: :- [#2] Calc(select=[a])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[j]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) :- Calc(select=[a]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- HashAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MAX(e) AS max$0]) +- Calc(select=[e], where=[(d > 0)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1253,15 +1253,15 @@ LogicalProject(b=[$1]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($1, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), >($1, $SCALAR_QUERY({ LogicalProject(EXPR$0=[*(0.5:DECIMAL(2, 1), $0)]) LogicalAggregate(group=[{}], agg#0=[SUM($0)]) LogicalProject(j=[$1]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1275,12 +1275,12 @@ Calc(select=[b]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_SUM(j) AS sum$0]) : +- Calc(select=[j], where=[(i < 100)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) :- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d], where=[(e > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1293,11 +1293,11 @@ Calc(select=[b]) LogicalProject(s=[$1]) +- LogicalFilter(condition=[AND(>($2, 2), IN($3, { LogicalProject(e=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) +- LogicalAggregate(group=[{0}], s=[SUM($1)], agg#1=[COUNT()], agg#2=[MAX($0)]) +- LogicalProject(b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1306,11 +1306,11 @@ Calc(select=[s]) +- MultipleInput(readOrder=[0,1], members=[\nNestedLoopJoin(joinType=[LeftSemiJoin], where=[($f3 = e)], select=[s, $f3], build=[right])\n:- Calc(select=[s, $f3], where=[($f2 > 2)])\n: +- HashAggregate(isMerge=[true], groupBy=[b], select=[b, Final_SUM(sum$0) AS s, Final_COUNT(count1$1) AS $f2, Final_MAX(max$2) AS $f3])\n: +- [#2] Exchange(distribution=[hash[b]])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[e]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[b]]) +- LocalHashAggregate(groupBy=[b], select=[b, Partial_SUM(a) AS sum$0, Partial_COUNT(*) AS count1$1, Partial_MAX(b) AS max$2]) +- Calc(select=[b, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) ]]> @@ -1328,13 +1328,13 @@ LogicalProject(c=[$2]) LogicalProject(d=[$0]) LogicalProject(d=[$0], e=[$1], f=[$2], i=[$3]) LogicalJoin(condition=[=($1, $4)], joinType=[full]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0], i0=[CAST($0):BIGINT]) LogicalProject(i=[$0]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1342,16 +1342,16 @@ LogicalProject(d=[$0]) Calc(select=[c]) +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, c], build=[right]) :- Calc(select=[a, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) +- NestedLoopJoin(joinType=[FullOuterJoin], where=[(e = i0)], select=[d, e, i0], build=[right]) :- Exchange(distribution=[single]) : +- Calc(select=[d, e]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[single]) +- Calc(select=[CAST(i AS BIGINT) AS i0], where=[(i > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1365,20 +1365,20 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($2, { LogicalProject(f1=[$3]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1394,9 +1394,9 @@ SELECT * FROM l WHERE LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN(+($0, 10), SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[+($0, 100)], EXPR$1=[SUBSTRING($2, 1, 5)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1404,10 +1404,10 @@ LogicalProject(EXPR$0=[+($0, 100)], EXPR$1=[SUBSTRING($2, 1, 5)]) Calc(select=[a, b, c]) +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[(($f3 = EXPR$0) AND ($f4 = EXPR$1))], select=[a, b, c, $f3, $f4], build=[right]) :- Calc(select=[a, b, c, (a + 10) AS $f3, SUBSTRING(c, 1, 5) AS $f4]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[(d + 100) AS EXPR$0, SUBSTRING(f, 1, 5) AS EXPR$1]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1423,15 +1423,15 @@ SELECT * FROM l WHERE (a, b) IN LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, $1, { LogicalProject(EXPR$0=[MAX($0) OVER ()], EXPR$1=[MIN($1) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1453,18 +1453,18 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[((a = $0) AND (b = $1))], select= LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1478,19 +1478,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), >($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d], where=[(e < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1503,9 +1503,9 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN(+($0, 1), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1513,10 +1513,10 @@ LogicalProject(d=[$0]) Calc(select=[a, b, c]) +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[($f3 = d)], select=[a, b, c, $f3], build=[right]) :- Calc(select=[a, b, c, (a + 1) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1534,12 +1534,12 @@ LogicalProject(a=[$0]) LogicalUnion(all=[false]) LogicalProject(e=[$1]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1547,16 +1547,16 @@ LogicalUnion(all=[false]) Calc(select=[a]) +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[(b = e)], select=[a, b], build=[right]) :- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- HashAggregate(isMerge=[true], groupBy=[e], select=[e]) +- Exchange(distribution=[hash[e]]) +- LocalHashAggregate(groupBy=[e], select=[e]) +- Union(all=[true], union=[e]) :- Calc(select=[e], where=[(d > 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Calc(select=[CAST(i AS BIGINT) AS i], where=[(i < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1569,28 +1569,28 @@ Calc(select=[a]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(EXISTS({ LogicalFilter(condition=[<>($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), EXISTS({ LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> d)], select=[a, b, c], build=[right])\n: :- [#2] LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) +MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], build=[right], singleRowJoin=[true])\n:- NestedLoopJoin(joinType=[LeftSemiJoin], where=[(a <> d)], select=[a, b, c], build=[right])\n: :- [#2] TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[], where=[m IS NOT NULL]) : +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) : +- Calc(select=[true AS i], where=[(j < 100)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) -:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) +:- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1603,28 +1603,28 @@ MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftSemiJoi LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[AND(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) +LogicalTableScan(table=[[default_catalog, default_database, t]]) }), =($cor0.a, $0))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1637,26 +1637,26 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), EXISTS({ LogicalFilter(condition=[>($0, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), >=($1, 1), EXISTS({ LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> = 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) ++- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1681,25 +1681,25 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) LogicalFilter(condition=[AND(=($cor0.a, $0), <>($2, _UTF-16LE'test'))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 'test')]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) -:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) +:- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, f]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1715,24 +1715,24 @@ LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), IN($2, { LogicalProject(k=[$2]) LogicalFilter(condition=[=($cor1.e, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor1]]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1745,25 +1745,25 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (b = e))], select=[a LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) LogicalFilter(condition=[=($2, $cor0.c)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1776,24 +1776,24 @@ MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftSemiJoi LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1808,25 +1808,25 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(d=[$0]) LogicalFilter(condition=[IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1839,29 +1839,29 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), NOT(EXISTS({ LogicalFilter(condition=[AND(=($cor0.b, $1), >(CAST($2):BIGINT, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), >=(CAST($2):BIGINT, 1), NOT(EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> 50)]) : +- Reused(reference_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a, b, c], where=[(CAST(c AS BIGINT) >= 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) ++- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1874,26 +1874,26 @@ MultipleInput(readOrder=[0,0,0,1], members=[\nNestedLoopJoin(joinType=[LeftAntiJ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), NOT(EXISTS({ LogicalFilter(condition=[>($0, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), >=($1, 1), NOT(EXISTS({ LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> = 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) ++- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1920,25 +1920,25 @@ LogicalProject(d=[$0]) LogicalFilter(condition=[NOT(IN($1, { LogicalProject(j=[$1]) LogicalFilter(condition=[=($0, $cor0.d)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1951,18 +1951,18 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[(a IS NULL OR d IS NULL OR (a = d LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[<($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1975,9 +1975,9 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[(a < d)], select=[a, b, c], build LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[<($cor0.a, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1985,11 +1985,11 @@ LogicalFilter(condition=[<($cor0.a, 10)]) Calc(select=[a, b, c]) +- NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f3], select=[a, b, c, $f3], build=[right]) :- Calc(select=[a, b, c, (a < 10) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- LocalHashAggregate(select=[]) +- Calc(select=[]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2002,18 +2002,18 @@ Calc(select=[a, b, c]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2025,9 +2025,9 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[(a = d)], select=[a, b, c], build ($1, 10), LIKE($2, _UTF-16LE'abc'), NOT(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) +LogicalTableScan(table=[[default_catalog, default_database, r]]) }))))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2035,14 +2035,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl Calc(select=[(a + 10) AS EXPR$0, c]) +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, c], build=[right], singleRowJoin=[true]) :- Calc(select=[a, c], where=[((b <= 10) AND NOT(LIKE(c, 'abc')))]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[], where=[m IS NOT NULL]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2054,22 +2054,22 @@ Calc(select=[(a + 10) AS EXPR$0, c]) @@ -2081,23 +2081,23 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c], build=[ri ($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[m IS NOT NULL AS $f0]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2112,21 +2112,21 @@ LogicalProject(a=[$0], b=[$1]) LogicalFilter(condition=[=($cor0.a, $0)]) LogicalAggregate(group=[{0}]) LogicalProject(c=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, rightT]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, leftT]]) ]]> @@ -2139,25 +2139,25 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[(a = c)], select=[a, b], build=[r LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })), NOT(IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2171,9 +2171,9 @@ LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) +- LogicalFilter(condition=[NOT(IN(SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=(+($cor0.b, 1), $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2181,10 +2181,10 @@ LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)]) Calc(select=[(a + 10) AS EXPR$0, c]) +- NestedLoopJoin(joinType=[LeftAntiJoin], where=[(($f3 IS NULL OR EXPR$0 IS NULL OR ($f3 = EXPR$0)) AND ($f4 = e))], select=[a, c, $f3, $f4], build=[right]) :- Calc(select=[a, c, SUBSTRING(c, 1, 5) AS $f3, (b + 1) AS $f4]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[SUBSTRING(f, 1, 5) AS EXPR$0, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2197,32 +2197,32 @@ Calc(select=[(a + 10) AS EXPR$0, c]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[NOT(IN(CASE(NOT(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), 1, NOT(IN($0, { LogicalProject(EXPR$0=[CAST($1):INTEGER]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), 2, 3), { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> = c0) AND a IS NOT NULL)), 1, ((c1 = 0) OR (i IS NULL AND (ck0 >= c1) AND a IS NOT NULL)), 2, 3) AS $f3])\n: +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = EXPR$0)], select=[a, b, c, c0, ck, i0, c1, ck0, EXPR$0, i], build=[right])\n: :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true])\n: : :- Calc(select=[a, b, c, c0, ck, i0])\n: : : +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, c0, ck, i, i0], build=[right])\n: : : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true])\n: : : : :- [#5] LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])\n: : : : +- [#6] Exchange(distribution=[broadcast])\n: : : +- [#4] Exchange(distribution=[broadcast])\n: : +- [#3] Exchange(distribution=[broadcast])\n: +- [#2] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) ++- MultipleInput(readOrder=[0,0,0,0,1,0], members=[\nNestedLoopJoin(joinType=[LeftAntiJoin], where=[((d IS NULL OR ($f3 = d)) AND (c = f))], select=[b, c, $f3], build=[right])\n:- Calc(select=[b, c, CASE(((c0 = 0) OR (i0 IS NULL AND (ck >= c0) AND a IS NOT NULL)), 1, ((c1 = 0) OR (i IS NULL AND (ck0 >= c1) AND a IS NOT NULL)), 2, 3) AS $f3])\n: +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = EXPR$0)], select=[a, b, c, c0, ck, i0, c1, ck0, EXPR$0, i], build=[right])\n: :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true])\n: : :- Calc(select=[a, b, c, c0, ck, i0])\n: : : +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, c0, ck, i, i0], build=[right])\n: : : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true])\n: : : : :- [#5] TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c])\n: : : : +- [#6] Exchange(distribution=[broadcast])\n: : : +- [#4] Exchange(distribution=[broadcast])\n: : +- [#3] Exchange(distribution=[broadcast])\n: +- [#2] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[d, f]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[EXPR$0, true AS i]) : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) : +- Exchange(distribution=[hash[EXPR$0]]) : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) : +- Calc(select=[CAST(j AS INTEGER) AS EXPR$0])(reuse_id=[1]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[2]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[2]) :- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : +- Exchange(distribution=[single]) @@ -2235,7 +2235,7 @@ Calc(select=[b]) : +- LocalHashAggregate(groupBy=[i], select=[i]) : +- Calc(select=[i])(reuse_id=[3]) : +- Reused(reference_id=[2]) - :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + :- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) +- Exchange(distribution=[single]) @@ -2254,18 +2254,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, $2, { LogicalProject(d=[$0], f=[$2]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2279,18 +2279,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> e))], select=[a, b, c], build=[right]) -:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +:- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2304,18 +2304,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($cor0.b, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10))], select=[a, b, c], build=[right]) -:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +:- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2329,18 +2329,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2354,19 +2354,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 1), NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, e], where=[(d > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2379,18 +2379,18 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[((a IS NULL OR d IS NULL OR (a = LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2403,19 +2403,19 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[(a IS NULL OR d IS NULL OR (a = d LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), LIKE($2, _UTF-16LE'abc'), NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10) AND LIKE(c, 'abc'))]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2428,15 +2428,15 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[(a IS NULL OR d IS NULL OR (a = d LogicalProject(b=[$1]) +- LogicalFilter(condition=[NOT(IN($1, CASE(NOT(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) })), 1, NOT(IN($0, { LogicalProject(j=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t2]]) })), 2, 3), { LogicalProject(e=[$1], d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2445,13 +2445,13 @@ Calc(select=[b]) +- MultipleInput(readOrder=[0,0,0,0,1,0], members=[\nNestedLoopJoin(joinType=[LeftAntiJoin], where=[((b IS NULL OR e IS NULL OR (b = e)) AND (d IS NULL OR ($f3 = d)))], select=[b, $f3], build=[right])\n:- Calc(select=[b, CASE(((c0 = 0) OR (i0 IS NULL AND (ck >= c0) AND a IS NOT NULL)), 1, ((c = 0) OR (i IS NULL AND (ck0 >= c) AND a IS NOT NULL)), 2, 3) AS $f3])\n: +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = j)], select=[a, b, c0, ck, i0, c, ck0, j, i], build=[right])\n: :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c0, ck, i0, c, ck0], build=[right], singleRowJoin=[true])\n: : :- Calc(select=[a, b, c AS c0, ck, i0])\n: : : +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, ck, i, i0], build=[right])\n: : : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, ck], build=[right], singleRowJoin=[true])\n: : : : :- [#5] Calc(select=[a, b])\n: : : : +- [#6] Exchange(distribution=[broadcast])\n: : : +- [#4] Exchange(distribution=[broadcast])\n: : +- [#3] Exchange(distribution=[broadcast])\n: +- [#2] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[e, d]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[j, true AS i]) : +- HashAggregate(isMerge=[true], groupBy=[j], select=[j]) : +- Exchange(distribution=[hash[j]]) : +- LocalHashAggregate(groupBy=[j], select=[j]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]], fields=[j])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, t2]], fields=[j])(reuse_id=[1]) :- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : +- Exchange(distribution=[single]) @@ -2462,9 +2462,9 @@ Calc(select=[b]) : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i]) : +- Exchange(distribution=[hash[i]]) : +- LocalHashAggregate(groupBy=[i], select=[i]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]], fields=[i])(reuse_id=[2]) + : +- TableSourceScan(table=[[default_catalog, default_database, t1]], fields=[i])(reuse_id=[2]) :- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) +- Exchange(distribution=[single]) @@ -2482,18 +2482,18 @@ Calc(select=[b]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, $2, { LogicalProject(d=[$0], f=[$2]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2506,18 +2506,18 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[((a IS NULL OR d IS NULL OR (a = LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2531,19 +2531,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })), >($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d], where=[(e < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2556,9 +2556,9 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[(a IS NULL OR d IS NULL OR (a = d LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN(*($0, $1), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2566,10 +2566,10 @@ LogicalProject(d=[$0]) Calc(select=[a, b, c]) +- NestedLoopJoin(joinType=[LeftAntiJoin], where=[($f3 IS NULL OR d IS NULL OR ($f3 = d))], select=[a, b, c, $f3], build=[right]) :- Calc(select=[a, b, c, (a * b) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml index 1b4724d477f17..ec20c141adffb 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml @@ -24,12 +24,12 @@ limitations under the License. @@ -37,17 +37,17 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) HashJoin(joinType=[LeftAntiJoin], where=[(a = i)], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[a]]) : +- NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true]) -: :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: :- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[broadcast]) : +- Calc(select=[m IS NOT NULL AS $f0]) : +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) : +- Calc(select=[true AS i]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[i]]) +- Calc(select=[i], where=[(j < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -62,23 +62,23 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$0=[MAX($4)]) LogicalProject(d=[$0], $f1=[true], f=[$2], $f3=[1], e=[$1]) LogicalFilter(condition=[AND(=($cor0.b, $1), <($0, 100), =($cor0.c, $2))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -92,21 +92,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=($cor1.a, $0)]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })], variablesSet=[[$cor1]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -119,18 +119,18 @@ HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[righ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[<($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -143,20 +143,20 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[(a < d)], select=[a, b, c], build LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[<($cor0.a, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -171,23 +171,23 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalAggregate(group=[{0}]) LogicalProject(e=[$1]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -200,19 +200,19 @@ HashJoin(joinType=[LeftSemiJoin], where=[(c = f)], select=[a, b, c], build=[righ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -224,9 +224,9 @@ HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[righ ($1, 10), NOT(OR(LIKE($2, _UTF-16LE'abc'), NOT(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) +LogicalTableScan(table=[[default_catalog, default_database, r]]) })))))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -234,14 +234,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl Calc(select=[(a + 10) AS EXPR$0, c]) +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, c], build=[right], singleRowJoin=[true]) :- Calc(select=[a, c], where=[((b > 10) AND NOT(LIKE(c, 'abc')))]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[], where=[m IS NOT NULL]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -254,19 +254,19 @@ Calc(select=[(a + 10) AS EXPR$0, c]) LogicalProject(c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalJoin(condition=[=($1, $3)], joinType=[full]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -292,19 +292,19 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[c], build=[right], LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) +LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -333,22 +333,22 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], build=[r @@ -360,23 +360,23 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], build=[r ($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[], where=[m IS NOT NULL]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -391,19 +391,19 @@ LogicalProject(a=[$0]) LogicalUnion(all=[false]) LogicalProject(e=[$1], f=[$2]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1], k=[$2]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Calc(select=[j, k], where=[(i < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -430,13 +430,13 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a], build=[right], LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -445,13 +445,13 @@ HashJoin(joinType=[LeftSemiJoin], where=[((a = i) AND (b = j))], select=[a, b, c :- Exchange(distribution=[hash[a, b]]) : +- HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[right]) : :- Exchange(distribution=[hash[a]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[i, j]]) +- Calc(select=[i, j]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -464,12 +464,12 @@ HashJoin(joinType=[LeftSemiJoin], where=[((a = i) AND (b = j))], select=[a, b, c LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -478,13 +478,13 @@ HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[righ :- Exchange(distribution=[hash[a]]) : +- HashJoin(joinType=[LeftSemiJoin], where=[(b = j)], select=[a, b, c], build=[right]) : :- Exchange(distribution=[hash[b]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[j]]) : +- Calc(select=[j]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -497,21 +497,21 @@ HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[righ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), NOT(IN($1, { LogicalProject(i=[$0]) LogicalFilter(condition=[AND(=($cor0.c, $2), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), EXISTS({ LogicalAggregate(group=[{0}], EXPR$0=[COUNT($0)]) LogicalProject(l=[$0]) LogicalFilter(condition=[LIKE($2, _UTF-16LE'Test')]) - LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t2]]) }), NOT(EXISTS({ LogicalFilter(condition=[<>($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -519,7 +519,7 @@ LogicalFilter(condition=[<>($cor0.b, $1)]) MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftAntiJoin], where=[(b <> e)], select=[a, b, c], build=[right])\n:- HashJoin(joinType=[LeftAntiJoin], where=[((b IS NULL OR i IS NULL OR (b = i)) AND (c = k))], select=[a, b, c], build=[right])\n: :- [#2] Exchange(distribution=[hash[c]])\n: +- [#3] Exchange(distribution=[hash[k]])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[e]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])(reuse_id=[1]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f])(reuse_id=[1]) :- Exchange(distribution=[hash[c]]) : +- MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], build=[right], singleRowJoin=[true])\n:- HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[right])\n: :- [#2] Exchange(distribution=[hash[a]])\n: +- [#3] Exchange(distribution=[hash[d]])\n+- [#1] Exchange(distribution=[broadcast])\n]) : :- Exchange(distribution=[broadcast]) @@ -532,15 +532,15 @@ MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftAntiJoi : : +- Exchange(distribution=[hash[l]]) : : +- LocalHashAggregate(groupBy=[l], select=[l]) : : +- Calc(select=[l], where=[LIKE(n, 'Test')]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]], fields=[l, m, n]) +: : +- TableSourceScan(table=[[default_catalog, default_database, t2]], fields=[l, m, n]) : :- Exchange(distribution=[hash[a]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) : +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[k]]) +- Calc(select=[i, k], where=[(i > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -556,23 +556,23 @@ LogicalProject(EXPR$0=[$1]) LogicalAggregate(group=[{0}], EXPR$0=[MAX($1)]) LogicalProject(f=[$2], e=[$1]) LogicalFilter(condition=[AND(=($cor0.c, $2), <($0, 3))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -588,22 +588,22 @@ LogicalProject(EXPR$0=[$4], d=[$0]) LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$0=[COUNT()]) LogicalProject(d=[$0], $f1=[true], e=[$1], $f3=[1]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -617,9 +617,9 @@ LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) +- LogicalFilter(condition=[IN(SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=(+($cor0.b, 1), $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -628,10 +628,10 @@ Calc(select=[(a + 10) AS EXPR$0, c]) +- HashJoin(joinType=[LeftSemiJoin], where=[(($f3 = EXPR$0) AND ($f4 = e))], select=[a, c, $f3, $f4], build=[right]) :- Exchange(distribution=[hash[$f3, $f4]]) : +- Calc(select=[a, c, SUBSTRING(c, 1, 5) AS $f3, (b + 1) AS $f4]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[EXPR$0, e]]) +- Calc(select=[SUBSTRING(f, 1, 5) AS EXPR$0, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -645,9 +645,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), NOT(LIKE($2, _UTF-16LE'abc')), IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -655,10 +655,10 @@ LogicalProject(d=[$0]) HashJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (b = e))], select=[a, b, c], build=[left]) :- Exchange(distribution=[hash[a, b]]) : +- Calc(select=[a, b, c], where=[((b > 10) AND NOT(LIKE(c, 'abc')))]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, e]]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -671,13 +671,13 @@ HashJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (b = e))], select=[a, b, c LogicalProject(b=[$1]) +- LogicalFilter(condition=[IN(CASE(IN($0, { LogicalProject(EXPR$0=[1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), 1, 2), { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -686,15 +686,15 @@ Calc(select=[b]) +- HashJoin(joinType=[LeftSemiJoin], where=[(($f3 = d) AND (c = f))], select=[b, c, $f3], build=[left]) :- Exchange(distribution=[hash[$f3, c]]) : +- Calc(select=[b, c, CASE(((c0 <> 0) AND i IS NOT NULL AND a IS NOT NULL), 1, 2) AS $f3]) - : +- MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = 1)], select=[a, b, c, c0, i], build=[right])\n:- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0], build=[right], singleRowJoin=[true])\n: :- [#2] LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) + : +- MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = 1)], select=[a, b, c, c0, i], build=[right])\n:- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0], build=[right], singleRowJoin=[true])\n: :- [#2] TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) : :- Exchange(distribution=[broadcast]) : : +- Calc(select=[true AS i]) : : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) : : +- Exchange(distribution=[hash[EXPR$0]]) : : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) : : +- Calc(select=[1 AS EXPR$0]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) - : :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) + : :- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c]) : +- Exchange(distribution=[single]) @@ -702,7 +702,7 @@ Calc(select=[b]) : +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[d, f]]) +- Calc(select=[d, f]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -718,12 +718,12 @@ LogicalProject(e=[$0]) LogicalJoin(condition=[=($0, $1)], joinType=[inner]) LogicalProject(e=[$1]) LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1]) LogicalFilter(condition=[AND(=($cor0.c, $2), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -731,16 +731,16 @@ LogicalProject(e=[$0]) Calc(select=[c]) +- HashJoin(joinType=[LeftSemiJoin], where=[((b = e) AND (a = d) AND (c = k))], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[b, a, c]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e, d, k]]) +- Calc(select=[e, d, k]) +- HashJoin(joinType=[InnerJoin], where=[(e = j)], select=[e, d, j, k], build=[left]) :- Exchange(distribution=[hash[e]]) : +- Calc(select=[e, d], where=[(e < 50)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j]]) +- Calc(select=[j, k], where=[(i > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -756,10 +756,10 @@ LogicalProject(j=[$3]) LogicalJoin(condition=[=($1, $4)], joinType=[left]) LogicalProject(e=[$1], f=[$2]) LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -767,16 +767,16 @@ LogicalProject(j=[$3]) Calc(select=[c]) +- HashJoin(joinType=[LeftSemiJoin], where=[((b = j) AND (a = d))], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[b, a]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[j, d]]) +- Calc(select=[j, d]) +- HashJoin(joinType=[LeftOuterJoin], where=[(f = k)], select=[f, d, j, k], build=[left]) :- Exchange(distribution=[hash[f]]) : +- Calc(select=[f, d], where=[(e < 50)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[k]]) +- Calc(select=[j, k]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -790,12 +790,12 @@ LogicalProject(c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalJoin(condition=[=($0, $3)], joinType=[right]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0]) LogicalFilter(condition=[AND(=($cor0.c, $2), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -804,16 +804,16 @@ Calc(select=[c]) +- HashJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (c = k))], select=[a, c], build=[right]) :- Exchange(distribution=[hash[a, c]]) : +- Calc(select=[a, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, k]]) +- Calc(select=[d, k]) +- HashJoin(joinType=[RightOuterJoin], where=[(d = i)], select=[d, i, k], build=[left]) :- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[i]]) +- Calc(select=[i, k], where=[(i > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -828,21 +828,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(f1=[$3]) LogicalFilter(condition=[=($cor1.a, $0)]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })], variablesSet=[[$cor1]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -856,9 +856,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, SUBSTRING($2, 1, 5), { LogicalProject(d=[$0], EXPR$1=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -867,10 +867,10 @@ Calc(select=[a, b, c]) +- HashJoin(joinType=[LeftSemiJoin], where=[((a = d) AND ($f3 = EXPR$1) AND (b = e))], select=[a, b, c, $f3], build=[right]) :- Exchange(distribution=[hash[a, $f3, b]]) : +- Calc(select=[a, b, c, SUBSTRING(c, 1, 5) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, EXPR$1, e]]) +- Calc(select=[d, SUBSTRING(f, 1, 5) AS EXPR$1, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -884,19 +884,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> e))], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[a]]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -910,9 +910,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($cor0.b, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -920,10 +920,10 @@ LogicalProject(d=[$0]) HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[(b > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -938,16 +938,16 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(rk=[$2]) LogicalFilter(condition=[<>($cor0.a, $0)]) LogicalProject(d=[$0], e=[$1], rk=[RANK() OVER (PARTITION BY $0 ORDER BY $1 NULLS FIRST)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> d))], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[b]]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[rk]]) +- Calc(select=[w0$o0 AS rk, d]) +- OverAggregate(partitionBy=[d], orderBy=[e ASC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[d, e, w0$o0]) @@ -955,7 +955,7 @@ HashJoin(joinType=[LeftSemiJoin], where=[((b = rk) AND (a <> d))], select=[a, b, +- Sort(orderBy=[d ASC, e ASC]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -971,15 +971,15 @@ LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) LogicalProject(d=[$0]) LogicalFilter(condition=[IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), { LogicalProject(i=[$0]) LogicalFilter(condition=[=($2, $cor0.c)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -989,7 +989,7 @@ Calc(select=[a]) :- Exchange(distribution=[hash[EXPR$0, c]]) : +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[a, c, EXPR$0], build=[right], singleRowJoin=[true]) : :- Calc(select=[a, c]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0]) : +- Exchange(distribution=[single]) @@ -998,11 +998,11 @@ Calc(select=[a]) : +- MultipleInput(readOrder=[1,0], members=[\nHashJoin(joinType=[InnerJoin], where=[(e = j)], select=[d, e, j], build=[right])\n:- [#1] Exchange(distribution=[hash[e]])\n+- HashAggregate(isMerge=[true], groupBy=[j], select=[j])\n +- [#2] Exchange(distribution=[hash[j]])\n]) : :- Exchange(distribution=[hash[e]]) : : +- Calc(select=[d, e]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) : +- Exchange(distribution=[hash[j]]) : +- LocalHashAggregate(groupBy=[j], select=[j]) : +- Calc(select=[j]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) +- Exchange(distribution=[hash[i, k]]) +- Calc(select=[i, k]) +- Reused(reference_id=[1]) @@ -1019,19 +1019,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1045,9 +1045,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 1), IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1055,10 +1055,10 @@ LogicalProject(d=[$0]) HashJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (b = e))], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[a, b]]) : +- Calc(select=[a, b, c], where=[(b > 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, e]]) +- Calc(select=[d, e], where=[(d > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1072,9 +1072,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=(CAST($cor0.b):INTEGER, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1082,10 +1082,10 @@ LogicalProject(d=[$0]) HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[left]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[(a = CAST(b AS INTEGER))]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1101,9 +1101,9 @@ LogicalProject(EXPR$0=[$1]) LogicalAggregate(group=[{0}], EXPR$0=[MAX($1)]) LogicalProject(f=[$2], e=[$1]) LogicalFilter(condition=[<($0, 3)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1112,14 +1112,14 @@ Calc(select=[a]) +- HashJoin(joinType=[LeftSemiJoin], where=[(b = EXPR$0)], select=[a, b], build=[right]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[EXPR$0]]) +- Calc(select=[EXPR$0]) +- HashAggregate(isMerge=[true], groupBy=[f], select=[f, Final_MAX(max$0) AS EXPR$0]) +- Exchange(distribution=[hash[f]]) +- LocalHashAggregate(groupBy=[f], select=[f, Partial_MAX(e) AS max$0]) +- Calc(select=[f, e], where=[(d < 3)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1133,22 +1133,22 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, $1, { LogicalAggregate(group=[{0}], EXPR$1=[COUNT()]) LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1161,9 +1161,9 @@ HashJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (b = EXPR$1))], select=[a, LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), NOT(LIKE($2, _UTF-16LE'abc')), IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1171,10 +1171,10 @@ LogicalProject(d=[$0]) HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[((b > 10) AND NOT(LIKE(c, 'abc')))]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1187,19 +1187,19 @@ HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[righ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IS TRUE(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1215,12 +1215,12 @@ SELECT b FROM l WHERE (CASE WHEN a IN LogicalProject(b=[$1]) +- LogicalFilter(condition=[IN(CASE(IN($0, { LogicalProject(EXPR$0=[1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), 1, 2), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1236,9 +1236,9 @@ Calc(select=[b]) : : +- Exchange(distribution=[hash[EXPR$0]]) : : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) : : +- Calc(select=[1 AS EXPR$0]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) + : : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) : :- Calc(select=[a, b]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c]) : +- Exchange(distribution=[single]) @@ -1246,7 +1246,7 @@ Calc(select=[b]) : +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1261,12 +1261,12 @@ LogicalProject(a=[$0]) LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) LogicalProject(e=[$1]) LogicalFilter(condition=[>($0, 0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1276,16 +1276,16 @@ Calc(select=[a]) :- Exchange(distribution=[hash[EXPR$0]]) : +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[a, EXPR$0], build=[right], singleRowJoin=[true]) : :- Calc(select=[a]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_MAX(e) AS max$0]) : +- Calc(select=[e], where=[(d > 0)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j]]) +- Calc(select=[j]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1299,15 +1299,15 @@ LogicalProject(b=[$1]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($1, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), >($1, $SCALAR_QUERY({ LogicalProject(EXPR$0=[*(0.5:DECIMAL(2, 1), $0)]) LogicalAggregate(group=[{}], agg#0=[SUM($0)]) LogicalProject(j=[$1]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1321,13 +1321,13 @@ Calc(select=[b]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_SUM(j) AS sum$0]) : +- Calc(select=[j], where=[(i < 100)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d], where=[(e > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1340,11 +1340,11 @@ Calc(select=[b]) LogicalProject(s=[$1]) +- LogicalFilter(condition=[AND(>($2, 2), IN($3, { LogicalProject(e=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) +- LogicalAggregate(group=[{0}], s=[SUM($1)], agg#1=[COUNT()], agg#2=[MAX($0)]) +- LogicalProject(b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1357,10 +1357,10 @@ Calc(select=[s]) : +- Exchange(distribution=[hash[b]]) : +- LocalHashAggregate(groupBy=[b], select=[b, Partial_SUM(a) AS sum$0, Partial_COUNT(*) AS count1$1, Partial_MAX(b) AS max$2]) : +- Calc(select=[b, a]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1378,13 +1378,13 @@ LogicalProject(c=[$2]) LogicalProject(d=[$0]) LogicalProject(d=[$0], e=[$1], f=[$2], i=[$3]) LogicalJoin(condition=[=($1, $4)], joinType=[full]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0], i0=[CAST($0):BIGINT]) LogicalProject(i=[$0]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1393,16 +1393,16 @@ Calc(select=[c]) +- HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, c], build=[right]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) +- HashJoin(joinType=[FullOuterJoin], where=[(e = i0)], select=[d, e, i0], build=[right]) :- Exchange(distribution=[hash[e]]) : +- Calc(select=[d, e]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[i0]]) +- Calc(select=[CAST(i AS BIGINT) AS i0], where=[(i > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1416,21 +1416,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($2, { LogicalProject(f1=[$3]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1446,9 +1446,9 @@ SELECT * FROM l WHERE LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN(+($0, 10), SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[+($0, 100)], EXPR$1=[SUBSTRING($2, 1, 5)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1457,10 +1457,10 @@ Calc(select=[a, b, c]) +- HashJoin(joinType=[LeftSemiJoin], where=[(($f3 = EXPR$0) AND ($f4 = EXPR$1))], select=[a, b, c, $f3, $f4], build=[right]) :- Exchange(distribution=[hash[$f3, $f4]]) : +- Calc(select=[a, b, c, (a + 10) AS $f3, SUBSTRING(c, 1, 5) AS $f4]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[EXPR$0, EXPR$1]]) +- Calc(select=[(d + 100) AS EXPR$0, SUBSTRING(f, 1, 5) AS EXPR$1]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1476,16 +1476,16 @@ SELECT * FROM l WHERE (a, b) IN LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, $1, { LogicalProject(EXPR$0=[MAX($0) OVER ()], EXPR$1=[MIN($1) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1507,19 +1507,19 @@ HashJoin(joinType=[LeftSemiJoin], where=[((a = $0) AND (b = $1))], select=[a, b, LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1533,9 +1533,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), >($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1543,10 +1543,10 @@ LogicalProject(d=[$0]) HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[(b > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d], where=[(e < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1559,9 +1559,9 @@ HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[righ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN(+($0, 1), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1570,10 +1570,10 @@ Calc(select=[a, b, c]) +- HashJoin(joinType=[LeftSemiJoin], where=[($f3 = d)], select=[a, b, c, $f3], build=[right]) :- Exchange(distribution=[hash[$f3]]) : +- Calc(select=[a, b, c, (a + 1) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1591,12 +1591,12 @@ LogicalProject(a=[$0]) LogicalUnion(all=[false]) LogicalProject(e=[$1]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1605,14 +1605,14 @@ Calc(select=[a]) +- MultipleInput(readOrder=[1,0], members=[\nHashJoin(joinType=[LeftSemiJoin], where=[(b = e)], select=[a, b], build=[right])\n:- [#1] Exchange(distribution=[hash[b]])\n+- HashAggregate(isMerge=[true], groupBy=[e], select=[e])\n +- [#2] Exchange(distribution=[hash[e]])\n]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- LocalHashAggregate(groupBy=[e], select=[e]) +- Union(all=[true], union=[e]) :- Calc(select=[e], where=[(d > 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Calc(select=[CAST(i AS BIGINT) AS i], where=[(i < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1625,28 +1625,28 @@ Calc(select=[a]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(EXISTS({ LogicalFilter(condition=[<>($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), EXISTS({ LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> d)], select=[a, b, c], build=[right])\n: :- [#2] LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) +MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], build=[right], singleRowJoin=[true])\n:- NestedLoopJoin(joinType=[LeftSemiJoin], where=[(a <> d)], select=[a, b, c], build=[right])\n: :- [#2] TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[], where=[m IS NOT NULL]) : +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) : +- Calc(select=[true AS i], where=[(j < 100)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) -:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) +:- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1659,29 +1659,29 @@ MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftSemiJoi LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[AND(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) +LogicalTableScan(table=[[default_catalog, default_database, t]]) }), =($cor0.a, $0))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1694,14 +1694,14 @@ HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[righ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), EXISTS({ LogicalFilter(condition=[>($0, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), >=($1, 1), EXISTS({ LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1713,7 +1713,7 @@ MultipleInput(readOrder=[0,0,0,1], members=[\nNestedLoopJoin(joinType=[LeftSemiJ : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) : +- Calc(select=[true AS i], where=[(j < 100)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[true AS $f0], where=[m IS NOT NULL]) : +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) @@ -1723,9 +1723,9 @@ MultipleInput(readOrder=[0,0,0,1], members=[\nNestedLoopJoin(joinType=[LeftSemiJ : +- Reused(reference_id=[1]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[(b >= 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1739,13 +1739,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) LogicalFilter(condition=[AND(=($cor0.a, $0), <>($2, _UTF-16LE'test'))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1754,13 +1754,13 @@ HashJoin(joinType=[LeftSemiJoin], where=[((b = j) AND (a = i))], select=[a, b, c :- Exchange(distribution=[hash[b, a]]) : +- HashJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (c = f))], select=[a, b, c], build=[right]) : :- Exchange(distribution=[hash[a, c]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d, f]]) : +- Calc(select=[d, f]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j, i]]) +- Calc(select=[j, i], where=[(k <> 'test')]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1776,26 +1776,26 @@ LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), IN($2, { LogicalProject(k=[$2]) LogicalFilter(condition=[=($cor1.e, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor1]]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1808,13 +1808,13 @@ HashJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (b = e))], select=[a, b, c LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) LogicalFilter(condition=[=($2, $cor0.c)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1823,13 +1823,13 @@ HashJoin(joinType=[LeftSemiJoin], where=[((b = j) AND (k = c))], select=[a, b, c :- Exchange(distribution=[hash[b, c]]) : +- HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[right]) : :- Exchange(distribution=[hash[a]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j, k]]) +- Calc(select=[j, k]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1842,12 +1842,12 @@ HashJoin(joinType=[LeftSemiJoin], where=[((b = j) AND (k = c))], select=[a, b, c LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1856,13 +1856,13 @@ HashJoin(joinType=[LeftSemiJoin], where=[(b = j)], select=[a, b, c], build=[righ :- Exchange(distribution=[hash[b]]) : +- HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[right]) : :- Exchange(distribution=[hash[a]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j]]) +- Calc(select=[j]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1877,27 +1877,27 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(d=[$0]) LogicalFilter(condition=[IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1910,14 +1910,14 @@ HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[righ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), NOT(EXISTS({ LogicalFilter(condition=[AND(=($cor0.b, $1), >(CAST($2):BIGINT, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), >=(CAST($2):BIGINT, 1), NOT(EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1929,12 +1929,12 @@ HashJoin(joinType=[LeftAntiJoin], where=[(a = i)], select=[a, b, c, d, e, f], bu : : +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], build=[left]) : : :- Exchange(distribution=[hash[a]]) : : : +- Calc(select=[a, b, c], where=[(CAST(c AS BIGINT) >= 1)]) -: : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : : +- Exchange(distribution=[hash[d]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) : +- Exchange(distribution=[hash[j]]) : +- Calc(select=[j], where=[(CAST(k AS BIGINT) > 50)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) +- Exchange(distribution=[hash[i]]) +- Calc(select=[i], where=[(j < 100)]) +- Reused(reference_id=[1]) @@ -1950,14 +1950,14 @@ HashJoin(joinType=[LeftAntiJoin], where=[(a = i)], select=[a, b, c, d, e, f], bu LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), NOT(EXISTS({ LogicalFilter(condition=[>($0, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), >=($1, 1), NOT(EXISTS({ LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1969,7 +1969,7 @@ MultipleInput(readOrder=[0,0,0,1], members=[\nNestedLoopJoin(joinType=[LeftAntiJ : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) : +- Calc(select=[true AS i], where=[(j < 100)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[m IS NOT NULL AS $f0]) : +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) @@ -1979,9 +1979,9 @@ MultipleInput(readOrder=[0,0,0,1], members=[\nNestedLoopJoin(joinType=[LeftAntiJ : +- Reused(reference_id=[1]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[(b >= 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1997,26 +1997,26 @@ LogicalProject(d=[$0]) LogicalFilter(condition=[NOT(IN($1, { LogicalProject(j=[$1]) LogicalFilter(condition=[=($0, $cor0.d)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2029,18 +2029,18 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[(a IS NULL OR d IS NULL OR (a = d LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[<($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2053,9 +2053,9 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[(a < d)], select=[a, b, c], build LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[<($cor0.a, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2063,11 +2063,11 @@ LogicalFilter(condition=[<($cor0.a, 10)]) Calc(select=[a, b, c]) +- NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f3], select=[a, b, c, $f3], build=[right]) :- Calc(select=[a, b, c, (a < 10) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- LocalHashAggregate(select=[]) +- Calc(select=[]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2080,19 +2080,19 @@ Calc(select=[a, b, c]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2104,9 +2104,9 @@ HashJoin(joinType=[LeftAntiJoin], where=[(a = d)], select=[a, b, c], build=[righ ($1, 10), LIKE($2, _UTF-16LE'abc'), NOT(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) +LogicalTableScan(table=[[default_catalog, default_database, r]]) }))))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2114,14 +2114,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl Calc(select=[(a + 10) AS EXPR$0, c]) +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, c], build=[right], singleRowJoin=[true]) :- Calc(select=[a, c], where=[((b <= 10) AND NOT(LIKE(c, 'abc')))]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[], where=[m IS NOT NULL]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2133,22 +2133,22 @@ Calc(select=[(a + 10) AS EXPR$0, c]) @@ -2160,23 +2160,23 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c], build=[ri ($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[m IS NOT NULL AS $f0]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2191,20 +2191,20 @@ LogicalProject(a=[$0], b=[$1]) LogicalFilter(condition=[=($cor0.a, $0)]) LogicalAggregate(group=[{0}]) LogicalProject(c=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, rightT]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, leftT]]) ]]> @@ -2217,13 +2217,13 @@ MultipleInput(readOrder=[1,0], members=[\nHashJoin(joinType=[LeftAntiJoin], wher LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })), NOT(IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2232,13 +2232,13 @@ HashJoin(joinType=[LeftAntiJoin], where=[((a IS NULL OR i IS NULL OR (a = i)) AN :- Exchange(distribution=[hash[b]]) : +- HashJoin(joinType=[LeftAntiJoin], where=[(a = d)], select=[a, b, c], build=[right]) : :- Exchange(distribution=[hash[a]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j]]) +- Calc(select=[i, j]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -2252,9 +2252,9 @@ LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) +- LogicalFilter(condition=[NOT(IN(SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=(+($cor0.b, 1), $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2263,10 +2263,10 @@ Calc(select=[(a + 10) AS EXPR$0, c]) +- HashJoin(joinType=[LeftAntiJoin], where=[(($f3 IS NULL OR EXPR$0 IS NULL OR ($f3 = EXPR$0)) AND ($f4 = e))], select=[a, c, $f3, $f4], build=[right]) :- Exchange(distribution=[hash[$f4]]) : +- Calc(select=[a, c, SUBSTRING(c, 1, 5) AS $f3, (b + 1) AS $f4]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[SUBSTRING(f, 1, 5) AS EXPR$0, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2279,16 +2279,16 @@ Calc(select=[(a + 10) AS EXPR$0, c]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[NOT(IN(CASE(NOT(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), 1, NOT(IN($0, { LogicalProject(EXPR$0=[CAST($1):INTEGER]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), 2, 3), { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2303,12 +2303,12 @@ Calc(select=[b]) : : +- Exchange(distribution=[single]) : : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1]) : : +- Calc(select=[CAST(j AS INTEGER) AS EXPR$0])(reuse_id=[1]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[2]) + : : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[2]) : :- Exchange(distribution=[hash[EXPR$0]]) : : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) : : +- Reused(reference_id=[1]) : :- Exchange(distribution=[hash[a]]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : :- Exchange(distribution=[broadcast]) : : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : : +- Exchange(distribution=[single]) @@ -2320,7 +2320,7 @@ Calc(select=[b]) : +- Reused(reference_id=[3]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[d, f]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2334,19 +2334,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, $2, { LogicalProject(d=[$0], f=[$2]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2360,18 +2360,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> e))], select=[a, b, c], build=[right]) -:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +:- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2385,18 +2385,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($cor0.b, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10))], select=[a, b, c], build=[right]) -:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +:- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2410,19 +2410,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2436,9 +2436,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 1), NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2446,10 +2446,10 @@ LogicalProject(d=[$0]) HashJoin(joinType=[LeftAntiJoin], where=[((a IS NULL OR d IS NULL OR (a = d)) AND (b = e))], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[a, b, c], where=[(b > 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[d, e], where=[(d > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2462,18 +2462,18 @@ HashJoin(joinType=[LeftAntiJoin], where=[((a IS NULL OR d IS NULL OR (a = d)) AN LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2486,19 +2486,19 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[(a IS NULL OR d IS NULL OR (a = d LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), LIKE($2, _UTF-16LE'abc'), NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10) AND LIKE(c, 'abc'))]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2511,15 +2511,15 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[(a IS NULL OR d IS NULL OR (a = d LogicalProject(b=[$1]) +- LogicalFilter(condition=[NOT(IN($1, CASE(NOT(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) })), 1, NOT(IN($0, { LogicalProject(j=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t2]]) })), 2, 3), { LogicalProject(e=[$1], d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2528,23 +2528,23 @@ Calc(select=[b]) +- MultipleInput(readOrder=[0,0,0,1,0,0], members=[\nNestedLoopJoin(joinType=[LeftAntiJoin], where=[((b IS NULL OR e IS NULL OR (b = e)) AND (d IS NULL OR ($f3 = d)))], select=[b, $f3], build=[right])\n:- Calc(select=[b, CASE(((c0 = 0) OR (i0 IS NULL AND (ck >= c0) AND a IS NOT NULL)), 1, ((c = 0) OR (i IS NULL AND (ck0 >= c) AND a IS NOT NULL)), 2, 3) AS $f3])\n: +- HashJoin(joinType=[LeftOuterJoin], where=[(a = j)], select=[a, b, c0, ck, i0, c, ck0, j, i], build=[right])\n: :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c0, ck, i0, c, ck0], build=[right], singleRowJoin=[true])\n: : :- Calc(select=[a, b, c AS c0, ck, i0])\n: : : +- HashJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, ck, i, i0], build=[right])\n: : : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, ck], build=[right], singleRowJoin=[true])\n: : : : :- [#4] Exchange(distribution=[hash[a]])\n: : : : +- [#5] Exchange(distribution=[broadcast])\n: : : +- Calc(select=[i, true AS i0])\n: : : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i])\n: : : +- [#6] Exchange(distribution=[hash[i]])\n: : +- [#2] Exchange(distribution=[broadcast])\n: +- Calc(select=[j, true AS i])\n: +- HashAggregate(isMerge=[true], groupBy=[j], select=[j])\n: +- [#3] Exchange(distribution=[hash[j]])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[e, d]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) :- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(j) AS count$1]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]], fields=[j])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, t2]], fields=[j])(reuse_id=[1]) :- Exchange(distribution=[hash[j]]) : +- LocalHashAggregate(groupBy=[j], select=[j]) : +- Reused(reference_id=[1]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) :- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(i) AS count$1]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]], fields=[i])(reuse_id=[2]) + : +- TableSourceScan(table=[[default_catalog, default_database, t1]], fields=[i])(reuse_id=[2]) +- Exchange(distribution=[hash[i]]) +- LocalHashAggregate(groupBy=[i], select=[i]) +- Reused(reference_id=[2]) @@ -2560,18 +2560,18 @@ Calc(select=[b]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, $2, { LogicalProject(d=[$0], f=[$2]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2584,18 +2584,18 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[((a IS NULL OR d IS NULL OR (a = LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2609,19 +2609,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })), >($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d], where=[(e < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2634,9 +2634,9 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[(a IS NULL OR d IS NULL OR (a = d LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN(*($0, $1), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2644,10 +2644,10 @@ LogicalProject(d=[$0]) Calc(select=[a, b, c]) +- NestedLoopJoin(joinType=[LeftAntiJoin], where=[($f3 IS NULL OR d IS NULL OR ($f3 = d))], select=[a, b, c, $f3], build=[right]) :- Calc(select=[a, b, c, (a * b) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2666,9 +2666,9 @@ LogicalProject(a=[$0]) +- LogicalFilter(condition=[NOT(IN($2, { LogicalProject(f=[$2]) LogicalFilter(condition=[=($2, $cor0.c)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2677,10 +2677,10 @@ Calc(select=[a]) +- HashJoin(joinType=[LeftAntiJoin], where=[AND(OR(IS NULL(c), IS NULL(f), =(c, f)), =(f, c))], select=[a, c], build=[right]) :- Exchange(distribution=[hash[c]]) : +- Calc(select=[a, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[f]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.xml index ee850e8890904..220f151f7e15e 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.xml @@ -24,8 +24,8 @@ limitations under the License. @@ -34,10 +34,10 @@ Calc(select=[c, g]) +- HashJoin(joinType=[FullOuterJoin], where=[((a = d) AND (d < 2))], select=[d, g, a, c], build=[left]) :- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, g]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -49,17 +49,17 @@ Calc(select=[c, g]) @@ -71,8 +71,8 @@ HashJoin(joinType=[FullOuterJoin], where=[((a = d) AND (d < 2) AND (b < h))], se @@ -81,10 +81,10 @@ Calc(select=[c, g]) +- HashJoin(joinType=[FullOuterJoin], where=[(b = e)], select=[b, c, e, g], build=[left]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[b, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[e, g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -99,9 +99,9 @@ SELECT * FROM (SELECT * FROM MyTable1) FULL JOIN (SELECT * FROM MyTable3) USING LogicalProject(a=[COALESCE($0, $3)], b=[$1], c=[$2], b0=[$4], c0=[$5]) +- LogicalJoin(condition=[=($0, $3)], joinType=[full]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3]]) ]]> @@ -109,9 +109,9 @@ LogicalProject(a=[COALESCE($0, $3)], b=[$1], c=[$2], b0=[$4], c0=[$5]) Calc(select=[COALESCE(a, a0) AS a, b, c, b0, c0]) +- HashJoin(joinType=[FullOuterJoin], where=[(a = a0)], select=[a, b, c, a0, b0, c0], build=[left]) :- Exchange(distribution=[hash[a]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[a]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable3]], fields=[a, b, c]) ]]> @@ -123,8 +123,8 @@ Calc(select=[COALESCE(a, a0) AS a, b, c, b0, c0]) @@ -133,10 +133,10 @@ Calc(select=[c, g]) +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[d, g, a, c], build=[left]) :- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, g], where=[(d < 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, c], where=[(a < 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -148,17 +148,17 @@ Calc(select=[c, g]) @@ -170,8 +170,8 @@ HashJoin(joinType=[LeftOuterJoin], where=[((a = d) AND (d < 2) AND (b < h))], se @@ -179,10 +179,10 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7]) HashJoin(joinType=[InnerJoin], where=[((a = d) AND (b < h))], select=[d, e, f, g, h, a, b, c], build=[right]) :- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, e, f, g, h], where=[(d < 2)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, c], where=[(a < 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -195,8 +195,8 @@ HashJoin(joinType=[InnerJoin], where=[((a = d) AND (b < h))], select=[d, e, f, g LogicalProject(c=[$2], g=[$6]) +- LogicalFilter(condition=[=($0, $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -205,10 +205,10 @@ Calc(select=[c, g]) +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, c, d, g], build=[left]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -221,8 +221,8 @@ Calc(select=[c, g]) LogicalProject(c=[$7], g=[$3]) +- LogicalFilter(condition=[AND(=($5, $0), <($0, 2))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -231,10 +231,10 @@ Calc(select=[c, g]) +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[d, g, a, c], build=[left]) :- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, g], where=[(d < 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, c], where=[(a < 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -255,10 +255,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -270,13 +270,13 @@ Calc(select=[a, b, CAST(2 AS INTEGER) AS d, e]) : +- Exchange(distribution=[hash[a]]) : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) : +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, e]]) +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e]) +- Exchange(distribution=[hash[d]]) +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -289,8 +289,8 @@ Calc(select=[a, b, CAST(2 AS INTEGER) AS d, e]) LogicalProject(c=[$2], g=[$6]) +- LogicalFilter(condition=[AND(=($1, $4), =($0, $3))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -298,10 +298,10 @@ LogicalProject(c=[$2], g=[$6]) Calc(select=[c, g]) +- HashJoin(joinType=[InnerJoin], where=[((b = e) AND (a = d))], select=[a, b, c, d, e, g], build=[left]) :- Exchange(distribution=[hash[b, a]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[e, d]]) +- Calc(select=[d, e, g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -313,8 +313,8 @@ Calc(select=[c, g]) @@ -323,9 +323,9 @@ Calc(select=[c, g]) +- HashJoin(joinType=[InnerJoin], where=[((a = d) AND (b = e))], select=[d, e, g, a, b, c], build=[left]) :- Exchange(distribution=[hash[d, e]]) : +- Calc(select=[d, e, g]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a, b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -338,8 +338,8 @@ Calc(select=[c, g]) LogicalProject(a=[$0], d=[$3]) +- LogicalFilter(condition=[=(+($0, 1), $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -348,10 +348,10 @@ Calc(select=[a, d]) +- HashJoin(joinType=[InnerJoin], where=[($f3 = d)], select=[a, $f3, d], build=[right]) :- Exchange(distribution=[hash[$f3]]) : +- Calc(select=[a, (a + 1) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -393,10 +393,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[true], joinType=[left]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -408,13 +408,13 @@ Calc(select=[CAST(2 AS INTEGER) AS a, b, d, CAST(e AS BIGINT) AS e]) : +- Exchange(distribution=[hash[a]]) : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) : +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, e]]) +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e]) +- Exchange(distribution=[hash[d]]) +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -434,10 +434,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($0, 2), =($3, 1))], joinType=[left]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -446,11 +446,11 @@ MultipleInput(readOrder=[1,0], members=[\nHashJoin(joinType=[LeftOuterJoin], whe :- Exchange(distribution=[hash[a]]) : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) : +- Calc(select=[a, b]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -462,8 +462,8 @@ MultipleInput(readOrder=[1,0], members=[\nHashJoin(joinType=[LeftOuterJoin], whe @@ -472,10 +472,10 @@ Calc(select=[c, g]) +- HashJoin(joinType=[LeftOuterJoin], where=[((a = d) AND (d < 2))], select=[d, g, a, c], build=[left]) :- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, g]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -487,8 +487,8 @@ Calc(select=[c, g]) @@ -497,10 +497,10 @@ Calc(select=[c, g]) +- HashJoin(joinType=[RightOuterJoin], where=[(b = e)], select=[b, c, e, g], build=[left]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[b, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[e, g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -520,10 +520,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($2, 2), =($1, 1))], joinType=[inner]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -533,11 +533,11 @@ Calc(select=[a, 1 AS b, CAST(2 AS INTEGER) AS d, 1 AS e]) :- Exchange(distribution=[hash[a]]) : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) : +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -550,8 +550,8 @@ Calc(select=[a, 1 AS b, CAST(2 AS INTEGER) AS d, 1 AS e]) LogicalProject(d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(IS NULL($3), <($0, 12))]) +- LogicalJoin(condition=[=($0, $3)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -560,10 +560,10 @@ Calc(select=[null:INTEGER AS d, e, f], where=[d IS NULL]) +- HashJoin(joinType=[LeftOuterJoin], where=[(a = d)], select=[a, d, e, f], build=[left]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a], where=[(a < 12)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, e, f], where=[(d < 12)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -576,8 +576,8 @@ Calc(select=[null:INTEGER AS d, e, f], where=[d IS NULL]) LogicalProject(d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(<($3, 10), <($0, 12))]) +- LogicalJoin(condition=[=($0, $3)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -586,10 +586,10 @@ Calc(select=[d, e, f]) +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d, e, f], build=[left]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a], where=[(a < 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, e, f], where=[(d < 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -602,8 +602,8 @@ Calc(select=[d, e, f]) LogicalProject(d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[=($3, null)]) +- LogicalJoin(condition=[=($0, $3)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -629,10 +629,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[true], joinType=[right]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -644,13 +644,13 @@ Calc(select=[a, CAST(b AS BIGINT) AS b, CAST(2 AS INTEGER) AS d, e]) : +- Exchange(distribution=[hash[a]]) : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) : +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, e]]) +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e]) +- Exchange(distribution=[hash[d]]) +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -670,10 +670,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($2, 2), =($1, 1))], joinType=[right]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -682,11 +682,11 @@ MultipleInput(readOrder=[1,0], members=[\nHashJoin(joinType=[RightOuterJoin], wh :- Exchange(distribution=[hash[d]]) : +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) : +- Calc(select=[d, e]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -698,8 +698,8 @@ MultipleInput(readOrder=[1,0], members=[\nHashJoin(joinType=[RightOuterJoin], wh @@ -708,10 +708,10 @@ Calc(select=[c, g]) +- HashJoin(joinType=[LeftOuterJoin], where=[(b = e)], select=[b, c, e, g], build=[left]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[b, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[e, g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -723,8 +723,8 @@ Calc(select=[c, g]) @@ -732,9 +732,9 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7]) HashJoin(joinType=[RightOuterJoin], where=[((a = d) AND (b < h))], select=[d, e, f, g, h, a, b, c], build=[left]) :- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, e, f, g, h], where=[(d < 2)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -746,8 +746,8 @@ HashJoin(joinType=[RightOuterJoin], where=[((a = d) AND (b < h))], select=[d, e, @@ -756,10 +756,10 @@ Calc(select=[c, g]) +- HashJoin(joinType=[RightOuterJoin], where=[(a = d)], select=[d, g, a, c], build=[left]) :- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, g], where=[(d < 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -778,16 +778,16 @@ LogicalProject(k=[$0], v=[$1], k0=[$2], v0=[$3]) +- LogicalJoin(condition=[AND(=($0, $2), >($2, 10))], joinType=[left]) :- LogicalProject(k=[$0], v=[$1]) : +- LogicalFilter(condition=[=($0, 0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, src]]) +- LogicalProject(k=[$0], v=[$1]) +- LogicalFilter(condition=[=($0, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, src]]) ]]> @@ -799,14 +799,14 @@ Calc(select=[CAST(0 AS BIGINT) AS k, v, null:BIGINT AS k0, null:VARCHAR(21474836 @@ -818,14 +818,14 @@ Calc(select=[null:INTEGER AS d, null:BIGINT AS e, null:INTEGER AS f, null:VARCHA diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml index 75bc9b55c0b0b..8f29cd63d0841 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml @@ -24,12 +24,12 @@ limitations under the License. @@ -37,17 +37,17 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) HashJoin(joinType=[LeftAntiJoin], where=[(a = i)], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[a]]) : +- NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true]) -: :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: :- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[broadcast]) : +- Calc(select=[m IS NOT NULL AS $f0]) : +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) : +- Calc(select=[true AS i]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[i]]) +- Calc(select=[i], where=[(j < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -62,23 +62,23 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$0=[MAX($4)]) LogicalProject(d=[$0], $f1=[true], f=[$2], $f3=[1], e=[$1]) LogicalFilter(condition=[AND(=($cor0.b, $1), <($0, 100), =($cor0.c, $2))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -92,21 +92,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=($cor1.a, $0)]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })], variablesSet=[[$cor1]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -121,23 +121,23 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalAggregate(group=[{0}]) LogicalProject(e=[$1]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -150,19 +150,19 @@ HashJoin(joinType=[LeftSemiJoin], where=[(c = f)], select=[a, b, c], build=[righ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -174,9 +174,9 @@ HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[righ ($1, 10), NOT(OR(LIKE($2, _UTF-16LE'abc'), NOT(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) +LogicalTableScan(table=[[default_catalog, default_database, r]]) })))))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -184,14 +184,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl Calc(select=[(a + 10) AS EXPR$0, c]) +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, c], build=[right], singleRowJoin=[true]) :- Calc(select=[a, c], where=[((b > 10) AND NOT(LIKE(c, 'abc')))]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[], where=[m IS NOT NULL]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -204,19 +204,19 @@ Calc(select=[(a + 10) AS EXPR$0, c]) LogicalProject(c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalJoin(condition=[=($1, $3)], joinType=[full]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -242,19 +242,19 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[c], build=[right], LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) +LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -283,22 +283,22 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], build=[r @@ -310,23 +310,23 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], build=[r ($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[], where=[m IS NOT NULL]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -341,19 +341,19 @@ LogicalProject(a=[$0]) LogicalUnion(all=[false]) LogicalProject(e=[$1], f=[$2]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1], k=[$2]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Calc(select=[j, k], where=[(i < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -380,13 +380,13 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a], build=[right], LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -395,13 +395,13 @@ HashJoin(joinType=[LeftSemiJoin], where=[((a = i) AND (b = j))], select=[a, b, c :- Exchange(distribution=[hash[a, b]]) : +- HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[right]) : :- Exchange(distribution=[hash[a]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[i, j]]) +- Calc(select=[i, j]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -414,12 +414,12 @@ HashJoin(joinType=[LeftSemiJoin], where=[((a = i) AND (b = j))], select=[a, b, c LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -428,13 +428,13 @@ HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[righ :- Exchange(distribution=[hash[a]]) : +- HashJoin(joinType=[LeftSemiJoin], where=[(b = j)], select=[a, b, c], build=[right]) : :- Exchange(distribution=[hash[b]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[j]]) : +- Calc(select=[j]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -450,23 +450,23 @@ LogicalProject(EXPR$0=[$1]) LogicalAggregate(group=[{0}], EXPR$0=[MAX($1)]) LogicalProject(f=[$2], e=[$1]) LogicalFilter(condition=[AND(=($cor0.c, $2), <($0, 3))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -482,22 +482,22 @@ LogicalProject(EXPR$0=[$4], d=[$0]) LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$0=[COUNT()]) LogicalProject(d=[$0], $f1=[true], e=[$1], $f3=[1]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -511,9 +511,9 @@ LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) +- LogicalFilter(condition=[IN(SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=(+($cor0.b, 1), $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -522,10 +522,10 @@ Calc(select=[(a + 10) AS EXPR$0, c]) +- HashJoin(joinType=[LeftSemiJoin], where=[(($f3 = EXPR$0) AND ($f4 = e))], select=[a, c, $f3, $f4], build=[right]) :- Exchange(distribution=[hash[$f3, $f4]]) : +- Calc(select=[a, c, SUBSTRING(c, 1, 5) AS $f3, (b + 1) AS $f4]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[EXPR$0, e]]) +- Calc(select=[SUBSTRING(f, 1, 5) AS EXPR$0, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -539,9 +539,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), NOT(LIKE($2, _UTF-16LE'abc')), IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -549,10 +549,10 @@ LogicalProject(d=[$0]) HashJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (b = e))], select=[a, b, c], build=[left]) :- Exchange(distribution=[hash[a, b]]) : +- Calc(select=[a, b, c], where=[((b > 10) AND NOT(LIKE(c, 'abc')))]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, e]]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -568,12 +568,12 @@ LogicalProject(e=[$0]) LogicalJoin(condition=[=($0, $1)], joinType=[inner]) LogicalProject(e=[$1]) LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1]) LogicalFilter(condition=[AND(=($cor0.c, $2), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -581,16 +581,16 @@ LogicalProject(e=[$0]) Calc(select=[c]) +- HashJoin(joinType=[LeftSemiJoin], where=[((b = e) AND (a = d) AND (c = k))], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[b, a, c]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e, d, k]]) +- Calc(select=[e, d, k]) +- HashJoin(joinType=[InnerJoin], where=[(e = j)], select=[e, d, j, k], build=[left]) :- Exchange(distribution=[hash[e]]) : +- Calc(select=[e, d], where=[(e < 50)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j]]) +- Calc(select=[j, k], where=[(i > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -606,10 +606,10 @@ LogicalProject(j=[$3]) LogicalJoin(condition=[=($1, $4)], joinType=[left]) LogicalProject(e=[$1], f=[$2]) LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -617,16 +617,16 @@ LogicalProject(j=[$3]) Calc(select=[c]) +- HashJoin(joinType=[LeftSemiJoin], where=[((b = j) AND (a = d))], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[b, a]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[j, d]]) +- Calc(select=[j, d]) +- HashJoin(joinType=[LeftOuterJoin], where=[(f = k)], select=[f, d, j, k], build=[left]) :- Exchange(distribution=[hash[f]]) : +- Calc(select=[f, d], where=[(e < 50)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[k]]) +- Calc(select=[j, k]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -640,12 +640,12 @@ LogicalProject(c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalJoin(condition=[=($0, $3)], joinType=[right]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0]) LogicalFilter(condition=[AND(=($cor0.c, $2), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -654,16 +654,16 @@ Calc(select=[c]) +- HashJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (c = k))], select=[a, c], build=[right]) :- Exchange(distribution=[hash[a, c]]) : +- Calc(select=[a, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, k]]) +- Calc(select=[d, k]) +- HashJoin(joinType=[RightOuterJoin], where=[(d = i)], select=[d, i, k], build=[left]) :- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[i]]) +- Calc(select=[i, k], where=[(i > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -678,21 +678,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(f1=[$3]) LogicalFilter(condition=[=($cor1.a, $0)]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })], variablesSet=[[$cor1]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -706,9 +706,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, SUBSTRING($2, 1, 5), { LogicalProject(d=[$0], EXPR$1=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -717,10 +717,10 @@ Calc(select=[a, b, c]) +- HashJoin(joinType=[LeftSemiJoin], where=[((a = d) AND ($f3 = EXPR$1) AND (b = e))], select=[a, b, c, $f3], build=[right]) :- Exchange(distribution=[hash[a, $f3, b]]) : +- Calc(select=[a, b, c, SUBSTRING(c, 1, 5) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, EXPR$1, e]]) +- Calc(select=[d, SUBSTRING(f, 1, 5) AS EXPR$1, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -734,19 +734,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> e))], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[a]]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -760,9 +760,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($cor0.b, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -770,10 +770,10 @@ LogicalProject(d=[$0]) HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[(b > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -788,16 +788,16 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(rk=[$2]) LogicalFilter(condition=[<>($cor0.a, $0)]) LogicalProject(d=[$0], e=[$1], rk=[RANK() OVER (PARTITION BY $0 ORDER BY $1 NULLS FIRST)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> d))], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[b]]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[rk]]) +- Calc(select=[w0$o0 AS rk, d]) +- OverAggregate(partitionBy=[d], orderBy=[e ASC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[d, e, w0$o0]) @@ -805,7 +805,7 @@ HashJoin(joinType=[LeftSemiJoin], where=[((b = rk) AND (a <> d))], select=[a, b, +- Sort(orderBy=[d ASC, e ASC]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -821,15 +821,15 @@ LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) LogicalProject(d=[$0]) LogicalFilter(condition=[IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), { LogicalProject(i=[$0]) LogicalFilter(condition=[=($2, $cor0.c)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -839,7 +839,7 @@ Calc(select=[a]) :- Exchange(distribution=[hash[EXPR$0, c]]) : +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[a, c, EXPR$0], build=[right], singleRowJoin=[true]) : :- Calc(select=[a, c]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0]) : +- Exchange(distribution=[single]) @@ -848,11 +848,11 @@ Calc(select=[a]) : +- MultipleInput(readOrder=[1,0], members=[\nHashJoin(joinType=[InnerJoin], where=[(e = j)], select=[d, e, j], build=[right])\n:- [#1] Exchange(distribution=[hash[e]])\n+- HashAggregate(isMerge=[true], groupBy=[j], select=[j])\n +- [#2] Exchange(distribution=[hash[j]])\n]) : :- Exchange(distribution=[hash[e]]) : : +- Calc(select=[d, e]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) : +- Exchange(distribution=[hash[j]]) : +- LocalHashAggregate(groupBy=[j], select=[j]) : +- Calc(select=[j]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) +- Exchange(distribution=[hash[i, k]]) +- Calc(select=[i, k]) +- Reused(reference_id=[1]) @@ -869,19 +869,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -895,9 +895,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 1), IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -905,10 +905,10 @@ LogicalProject(d=[$0]) HashJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (b = e))], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[a, b]]) : +- Calc(select=[a, b, c], where=[(b > 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, e]]) +- Calc(select=[d, e], where=[(d > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -922,9 +922,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=(CAST($cor0.b):INTEGER, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -932,10 +932,10 @@ LogicalProject(d=[$0]) HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[left]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[(a = CAST(b AS INTEGER))]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -951,9 +951,9 @@ LogicalProject(EXPR$0=[$1]) LogicalAggregate(group=[{0}], EXPR$0=[MAX($1)]) LogicalProject(f=[$2], e=[$1]) LogicalFilter(condition=[<($0, 3)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -962,14 +962,14 @@ Calc(select=[a]) +- HashJoin(joinType=[LeftSemiJoin], where=[(b = EXPR$0)], select=[a, b], build=[right]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[EXPR$0]]) +- Calc(select=[EXPR$0]) +- HashAggregate(isMerge=[true], groupBy=[f], select=[f, Final_MAX(max$0) AS EXPR$0]) +- Exchange(distribution=[hash[f]]) +- LocalHashAggregate(groupBy=[f], select=[f, Partial_MAX(e) AS max$0]) +- Calc(select=[f, e], where=[(d < 3)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -983,22 +983,22 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, $1, { LogicalAggregate(group=[{0}], EXPR$1=[COUNT()]) LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1011,9 +1011,9 @@ HashJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (b = EXPR$1))], select=[a, LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), NOT(LIKE($2, _UTF-16LE'abc')), IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1021,10 +1021,10 @@ LogicalProject(d=[$0]) HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[((b > 10) AND NOT(LIKE(c, 'abc')))]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1037,19 +1037,19 @@ HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[righ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IS TRUE(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1064,12 +1064,12 @@ LogicalProject(a=[$0]) LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) LogicalProject(e=[$1]) LogicalFilter(condition=[>($0, 0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1079,16 +1079,16 @@ Calc(select=[a]) :- Exchange(distribution=[hash[EXPR$0]]) : +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[a, EXPR$0], build=[right], singleRowJoin=[true]) : :- Calc(select=[a]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_MAX(e) AS max$0]) : +- Calc(select=[e], where=[(d > 0)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j]]) +- Calc(select=[j]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1102,15 +1102,15 @@ LogicalProject(b=[$1]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($1, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), >($1, $SCALAR_QUERY({ LogicalProject(EXPR$0=[*(0.5:DECIMAL(2, 1), $0)]) LogicalAggregate(group=[{}], agg#0=[SUM($0)]) LogicalProject(j=[$1]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1124,13 +1124,13 @@ Calc(select=[b]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_SUM(j) AS sum$0]) : +- Calc(select=[j], where=[(i < 100)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d], where=[(e > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1143,11 +1143,11 @@ Calc(select=[b]) LogicalProject(s=[$1]) +- LogicalFilter(condition=[AND(>($2, 2), IN($3, { LogicalProject(e=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) +- LogicalAggregate(group=[{0}], s=[SUM($1)], agg#1=[COUNT()], agg#2=[MAX($0)]) +- LogicalProject(b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1160,10 +1160,10 @@ Calc(select=[s]) : +- Exchange(distribution=[hash[b]]) : +- LocalHashAggregate(groupBy=[b], select=[b, Partial_SUM(a) AS sum$0, Partial_COUNT(*) AS count1$1, Partial_MAX(b) AS max$2]) : +- Calc(select=[b, a]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1181,13 +1181,13 @@ LogicalProject(c=[$2]) LogicalProject(d=[$0]) LogicalProject(d=[$0], e=[$1], f=[$2], i=[$3]) LogicalJoin(condition=[=($1, $4)], joinType=[full]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0], i0=[CAST($0):BIGINT]) LogicalProject(i=[$0]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1196,16 +1196,16 @@ Calc(select=[c]) +- HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, c], build=[right]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) +- HashJoin(joinType=[FullOuterJoin], where=[(e = i0)], select=[d, e, i0], build=[right]) :- Exchange(distribution=[hash[e]]) : +- Calc(select=[d, e]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[i0]]) +- Calc(select=[CAST(i AS BIGINT) AS i0], where=[(i > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1219,21 +1219,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($2, { LogicalProject(f1=[$3]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1249,9 +1249,9 @@ SELECT * FROM l WHERE LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN(+($0, 10), SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[+($0, 100)], EXPR$1=[SUBSTRING($2, 1, 5)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1260,10 +1260,10 @@ Calc(select=[a, b, c]) +- HashJoin(joinType=[LeftSemiJoin], where=[(($f3 = EXPR$0) AND ($f4 = EXPR$1))], select=[a, b, c, $f3, $f4], build=[right]) :- Exchange(distribution=[hash[$f3, $f4]]) : +- Calc(select=[a, b, c, (a + 10) AS $f3, SUBSTRING(c, 1, 5) AS $f4]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[EXPR$0, EXPR$1]]) +- Calc(select=[(d + 100) AS EXPR$0, SUBSTRING(f, 1, 5) AS EXPR$1]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1279,16 +1279,16 @@ SELECT * FROM l WHERE (a, b) IN LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, $1, { LogicalProject(EXPR$0=[MAX($0) OVER ()], EXPR$1=[MIN($1) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1310,19 +1310,19 @@ HashJoin(joinType=[LeftSemiJoin], where=[((a = $0) AND (b = $1))], select=[a, b, LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1336,9 +1336,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), >($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1346,10 +1346,10 @@ LogicalProject(d=[$0]) HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[(b > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d], where=[(e < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1362,9 +1362,9 @@ HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[righ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN(+($0, 1), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1373,10 +1373,10 @@ Calc(select=[a, b, c]) +- HashJoin(joinType=[LeftSemiJoin], where=[($f3 = d)], select=[a, b, c, $f3], build=[right]) :- Exchange(distribution=[hash[$f3]]) : +- Calc(select=[a, b, c, (a + 1) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1394,12 +1394,12 @@ LogicalProject(a=[$0]) LogicalUnion(all=[false]) LogicalProject(e=[$1]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1408,14 +1408,14 @@ Calc(select=[a]) +- MultipleInput(readOrder=[1,0], members=[\nHashJoin(joinType=[LeftSemiJoin], where=[(b = e)], select=[a, b], build=[right])\n:- [#1] Exchange(distribution=[hash[b]])\n+- HashAggregate(isMerge=[true], groupBy=[e], select=[e])\n +- [#2] Exchange(distribution=[hash[e]])\n]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- LocalHashAggregate(groupBy=[e], select=[e]) +- Union(all=[true], union=[e]) :- Calc(select=[e], where=[(d > 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Calc(select=[CAST(i AS BIGINT) AS i], where=[(i < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1428,29 +1428,29 @@ Calc(select=[a]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[AND(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) +LogicalTableScan(table=[[default_catalog, default_database, t]]) }), =($cor0.a, $0))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1463,14 +1463,14 @@ HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[righ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), EXISTS({ LogicalFilter(condition=[>($0, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), >=($1, 1), EXISTS({ LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1482,7 +1482,7 @@ MultipleInput(readOrder=[0,0,0,1], members=[\nNestedLoopJoin(joinType=[LeftSemiJ : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) : +- Calc(select=[true AS i], where=[(j < 100)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[true AS $f0], where=[m IS NOT NULL]) : +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) @@ -1492,9 +1492,9 @@ MultipleInput(readOrder=[0,0,0,1], members=[\nNestedLoopJoin(joinType=[LeftSemiJ : +- Reused(reference_id=[1]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[(b >= 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1507,14 +1507,14 @@ MultipleInput(readOrder=[0,0,0,1], members=[\nNestedLoopJoin(joinType=[LeftSemiJ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), NOT(EXISTS({ LogicalFilter(condition=[AND(=($cor0.b, $1), >(CAST($2):BIGINT, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), >=(CAST($2):BIGINT, 1), NOT(EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1526,12 +1526,12 @@ HashJoin(joinType=[LeftAntiJoin], where=[(a = i)], select=[a, b, c, d, e, f], bu : : +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], build=[left]) : : :- Exchange(distribution=[hash[a]]) : : : +- Calc(select=[a, b, c], where=[(CAST(c AS BIGINT) >= 1)]) -: : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : : +- Exchange(distribution=[hash[d]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) : +- Exchange(distribution=[hash[j]]) : +- Calc(select=[j], where=[(CAST(k AS BIGINT) > 50)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) +- Exchange(distribution=[hash[i]]) +- Calc(select=[i], where=[(j < 100)]) +- Reused(reference_id=[1]) @@ -1548,13 +1548,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) LogicalFilter(condition=[AND(=($cor0.a, $0), <>($2, _UTF-16LE'test'))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1563,13 +1563,13 @@ HashJoin(joinType=[LeftSemiJoin], where=[((b = j) AND (a = i))], select=[a, b, c :- Exchange(distribution=[hash[b, a]]) : +- HashJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (c = f))], select=[a, b, c], build=[right]) : :- Exchange(distribution=[hash[a, c]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d, f]]) : +- Calc(select=[d, f]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j, i]]) +- Calc(select=[j, i], where=[(k <> 'test')]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1585,26 +1585,26 @@ LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), IN($2, { LogicalProject(k=[$2]) LogicalFilter(condition=[=($cor1.e, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor1]]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1617,13 +1617,13 @@ HashJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (b = e))], select=[a, b, c LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) LogicalFilter(condition=[=($2, $cor0.c)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1632,13 +1632,13 @@ HashJoin(joinType=[LeftSemiJoin], where=[((b = j) AND (k = c))], select=[a, b, c :- Exchange(distribution=[hash[b, c]]) : +- HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[right]) : :- Exchange(distribution=[hash[a]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j, k]]) +- Calc(select=[j, k]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1651,12 +1651,12 @@ HashJoin(joinType=[LeftSemiJoin], where=[((b = j) AND (k = c))], select=[a, b, c LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1665,13 +1665,13 @@ HashJoin(joinType=[LeftSemiJoin], where=[(b = j)], select=[a, b, c], build=[righ :- Exchange(distribution=[hash[b]]) : +- HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[right]) : :- Exchange(distribution=[hash[a]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j]]) +- Calc(select=[j]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1686,27 +1686,27 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(d=[$0]) LogicalFilter(condition=[IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1719,14 +1719,14 @@ HashJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], build=[righ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), NOT(EXISTS({ LogicalFilter(condition=[>($0, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), >=($1, 1), NOT(EXISTS({ LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1738,7 +1738,7 @@ MultipleInput(readOrder=[0,0,0,1], members=[\nNestedLoopJoin(joinType=[LeftAntiJ : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) : +- Calc(select=[true AS i], where=[(j < 100)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[m IS NOT NULL AS $f0]) : +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) @@ -1748,9 +1748,9 @@ MultipleInput(readOrder=[0,0,0,1], members=[\nNestedLoopJoin(joinType=[LeftAntiJ : +- Reused(reference_id=[1]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[(b >= 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1763,19 +1763,19 @@ MultipleInput(readOrder=[0,0,0,1], members=[\nNestedLoopJoin(joinType=[LeftAntiJ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1787,9 +1787,9 @@ HashJoin(joinType=[LeftAntiJoin], where=[(a = d)], select=[a, b, c], build=[righ ($1, 10), LIKE($2, _UTF-16LE'abc'), NOT(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) +LogicalTableScan(table=[[default_catalog, default_database, r]]) }))))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1797,14 +1797,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl Calc(select=[(a + 10) AS EXPR$0, c]) +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, c], build=[right], singleRowJoin=[true]) :- Calc(select=[a, c], where=[((b <= 10) AND NOT(LIKE(c, 'abc')))]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[], where=[m IS NOT NULL]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1816,22 +1816,22 @@ Calc(select=[(a + 10) AS EXPR$0, c]) @@ -1843,23 +1843,23 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c], build=[ri ($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[m IS NOT NULL AS $f0]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1874,20 +1874,20 @@ LogicalProject(a=[$0], b=[$1]) LogicalFilter(condition=[=($cor0.a, $0)]) LogicalAggregate(group=[{0}]) LogicalProject(c=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, rightT]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, leftT]]) ]]> @@ -1900,13 +1900,13 @@ MultipleInput(readOrder=[1,0], members=[\nHashJoin(joinType=[LeftAntiJoin], wher LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })), NOT(IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1915,13 +1915,13 @@ HashJoin(joinType=[LeftAntiJoin], where=[((a IS NULL OR i IS NULL OR (a = i)) AN :- Exchange(distribution=[hash[b]]) : +- HashJoin(joinType=[LeftAntiJoin], where=[(a = d)], select=[a, b, c], build=[right]) : :- Exchange(distribution=[hash[a]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j]]) +- Calc(select=[i, j]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1935,9 +1935,9 @@ LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) +- LogicalFilter(condition=[NOT(IN(SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=(+($cor0.b, 1), $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1946,10 +1946,10 @@ Calc(select=[(a + 10) AS EXPR$0, c]) +- HashJoin(joinType=[LeftAntiJoin], where=[(($f3 IS NULL OR EXPR$0 IS NULL OR ($f3 = EXPR$0)) AND ($f4 = e))], select=[a, c, $f3, $f4], build=[right]) :- Exchange(distribution=[hash[$f4]]) : +- Calc(select=[a, c, SUBSTRING(c, 1, 5) AS $f3, (b + 1) AS $f4]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[SUBSTRING(f, 1, 5) AS EXPR$0, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1962,16 +1962,16 @@ Calc(select=[(a + 10) AS EXPR$0, c]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[NOT(IN(CASE(NOT(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), 1, NOT(IN($0, { LogicalProject(EXPR$0=[CAST($1):INTEGER]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), 2, 3), { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1986,12 +1986,12 @@ Calc(select=[b]) : : +- Exchange(distribution=[single]) : : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1]) : : +- Calc(select=[CAST(j AS INTEGER) AS EXPR$0])(reuse_id=[1]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[2]) + : : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[2]) : :- Exchange(distribution=[hash[EXPR$0]]) : : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) : : +- Reused(reference_id=[1]) : :- Exchange(distribution=[hash[a]]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : :- Exchange(distribution=[broadcast]) : : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : : +- Exchange(distribution=[single]) @@ -2003,7 +2003,7 @@ Calc(select=[b]) : +- Reused(reference_id=[3]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[d, f]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2017,19 +2017,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, $2, { LogicalProject(d=[$0], f=[$2]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2043,19 +2043,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2069,9 +2069,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 1), NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2079,10 +2079,10 @@ LogicalProject(d=[$0]) HashJoin(joinType=[LeftAntiJoin], where=[((a IS NULL OR d IS NULL OR (a = d)) AND (b = e))], select=[a, b, c], build=[right]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[a, b, c], where=[(b > 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[d, e], where=[(d > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SingleRowJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SingleRowJoinTest.xml index e319e8209efa6..a63d51dc74b8d 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SingleRowJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SingleRowJoinTest.xml @@ -25,10 +25,10 @@ limitations under the License. LogicalProject(a1=[$0], a2=[$1]) +- LogicalFilter(condition=[<($0, $2)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalAggregate(group=[{}], cnt=[COUNT($0)]) +- LogicalProject(a1=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, A]]) ]]> @@ -36,7 +36,7 @@ LogicalProject(a1=[$0], a2=[$1]) Calc(select=[a1, a2]) +- NestedLoopJoin(joinType=[InnerJoin], where=[(a1 < cnt)], select=[a1, a2, cnt], build=[right], singleRowJoin=[true]) :- Exchange(distribution=[any], shuffle_mode=[BATCH]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]], fields=[a1, a2])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2])(reuse_id=[1]) +- Exchange(distribution=[broadcast]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS cnt]) +- Exchange(distribution=[single]) @@ -56,8 +56,8 @@ LogicalProject(a1=[$1]) +- LogicalJoin(condition=[=($0, $2)], joinType=[right]) :- LogicalAggregate(group=[{}], cnt=[COUNT()]) : +- LogicalProject($f0=[0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, B]]) + +- LogicalTableScan(table=[[default_catalog, default_database, A]]) ]]> @@ -69,8 +69,8 @@ Calc(select=[a1]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) : +- Calc(select=[0 AS $f0]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]], fields=[b1, b2]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]], fields=[a1, a2]) + : +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2]) + +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2]) ]]> @@ -84,8 +84,8 @@ LogicalProject(a1=[$1]) +- LogicalJoin(condition=[<($0, $2)], joinType=[right]) :- LogicalAggregate(group=[{}], cnt=[COUNT()]) : +- LogicalProject($f0=[0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, B]]) + +- LogicalTableScan(table=[[default_catalog, default_database, A]]) ]]> @@ -97,8 +97,8 @@ Calc(select=[a1]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) : +- Calc(select=[0 AS $f0]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]], fields=[b1, b2]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]], fields=[a1, a2]) + : +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2]) + +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2]) ]]> @@ -110,23 +110,23 @@ Calc(select=[a1]) ($0, $2)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalAggregate(group=[{}], cnt=[COUNT()]) +- LogicalProject($f0=[0]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> cnt)], select=[a1, a2, cnt], build=[right], singleRowJoin=[true]) - :- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]], fields=[a1, a2]) + :- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2]) +- Exchange(distribution=[broadcast]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS cnt]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) +- Calc(select=[0 AS $f0]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]], fields=[b1, b2]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2]) ]]> @@ -138,10 +138,10 @@ Calc(select=[a2]) @@ -149,7 +149,7 @@ LogicalProject(a1=[$0], a_sum=[$2]) NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a1, a_sum], build=[right], singleRowJoin=[true]) :- Exchange(distribution=[any], shuffle_mode=[BATCH]) : +- Calc(select=[a1]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]], fields=[a1, a2])(reuse_id=[1]) +: +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2])(reuse_id=[1]) +- Exchange(distribution=[broadcast]) +- Calc(select=[($f0 + $f1) AS a_sum]) +- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS $f0, Final_SUM(sum$1) AS $f1]) @@ -167,23 +167,23 @@ NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a1, a_sum], build=[ri @@ -200,20 +200,20 @@ WHERE a1 < b1 AND a2 = b2 LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3]) +- LogicalFilter(condition=[AND(<($0, $2), =($1, $3))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalAggregate(group=[{}], b1=[MIN($0)], b2=[MAX($1)]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -227,11 +227,11 @@ LogicalFilter(condition=[>($1, $SCALAR_QUERY({ LogicalProject(EXPR$0=[*($0, 0.1:DECIMAL(2, 1))]) LogicalAggregate(group=[{}], agg#0=[SUM($0)]) LogicalProject(a1=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) + LogicalTableScan(table=[[default_catalog, default_database, A]]) }))]) +- LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)]) +- LogicalProject(a2=[$1], a1=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, A]]) ]]> @@ -245,7 +245,7 @@ Calc(select=[a2, EXPR$1]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_SUM(a1) AS sum$0]) : +- Calc(select=[a1]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]], fields=[a1, a2])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2])(reuse_id=[1]) +- Exchange(distribution=[hash[a2]]) +- LocalHashAggregate(groupBy=[a2], select=[a2, Partial_SUM(a1) AS sum$0]) +- Reused(reference_id=[1]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.xml index ed26a26c4139c..5bc56c5b84628 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.xml @@ -24,8 +24,8 @@ limitations under the License. @@ -34,10 +34,10 @@ Calc(select=[c, g]) +- SortMergeJoin(joinType=[FullOuterJoin], where=[((a = d) AND (d < 2))], select=[d, g, a, c]) :- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, g]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -49,17 +49,17 @@ Calc(select=[c, g]) @@ -71,8 +71,8 @@ SortMergeJoin(joinType=[FullOuterJoin], where=[((a = d) AND (d < 2) AND (b < h)) @@ -81,10 +81,10 @@ Calc(select=[c, g]) +- SortMergeJoin(joinType=[FullOuterJoin], where=[(b = e)], select=[b, c, e, g]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[b, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[e, g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -99,9 +99,9 @@ SELECT * FROM (SELECT * FROM MyTable1) FULL JOIN (SELECT * FROM MyTable3) USING LogicalProject(a=[COALESCE($0, $3)], b=[$1], c=[$2], b0=[$4], c0=[$5]) +- LogicalJoin(condition=[=($0, $3)], joinType=[full]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3]]) ]]> @@ -109,9 +109,9 @@ LogicalProject(a=[COALESCE($0, $3)], b=[$1], c=[$2], b0=[$4], c0=[$5]) Calc(select=[COALESCE(a, a0) AS a, b, c, b0, c0]) +- SortMergeJoin(joinType=[FullOuterJoin], where=[(a = a0)], select=[a, b, c, a0, b0, c0]) :- Exchange(distribution=[hash[a]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[a]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable3]], fields=[a, b, c]) ]]> @@ -123,8 +123,8 @@ Calc(select=[COALESCE(a, a0) AS a, b, c, b0, c0]) @@ -133,10 +133,10 @@ Calc(select=[c, g]) +- SortMergeJoin(joinType=[InnerJoin], where=[(a = d)], select=[d, g, a, c]) :- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, g], where=[(d < 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, c], where=[(a < 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -148,17 +148,17 @@ Calc(select=[c, g]) @@ -170,8 +170,8 @@ SortMergeJoin(joinType=[LeftOuterJoin], where=[((a = d) AND (d < 2) AND (b < h)) @@ -179,10 +179,10 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7]) SortMergeJoin(joinType=[InnerJoin], where=[((a = d) AND (b < h))], select=[d, e, f, g, h, a, b, c]) :- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, e, f, g, h], where=[(d < 2)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, c], where=[(a < 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -195,8 +195,8 @@ SortMergeJoin(joinType=[InnerJoin], where=[((a = d) AND (b < h))], select=[d, e, LogicalProject(c=[$2], g=[$6]) +- LogicalFilter(condition=[=($0, $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -205,10 +205,10 @@ Calc(select=[c, g]) +- SortMergeJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, c, d, g]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -221,8 +221,8 @@ Calc(select=[c, g]) LogicalProject(c=[$7], g=[$3]) +- LogicalFilter(condition=[AND(=($5, $0), <($0, 2))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -231,10 +231,10 @@ Calc(select=[c, g]) +- SortMergeJoin(joinType=[InnerJoin], where=[(a = d)], select=[d, g, a, c]) :- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, g], where=[(d < 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, c], where=[(a < 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -255,10 +255,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -270,13 +270,13 @@ Calc(select=[a, b, CAST(2 AS INTEGER) AS d, e]) : +- Exchange(distribution=[hash[a]]) : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) : +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, e]]) +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e]) +- Exchange(distribution=[hash[d]]) +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -289,8 +289,8 @@ Calc(select=[a, b, CAST(2 AS INTEGER) AS d, e]) LogicalProject(c=[$2], g=[$6]) +- LogicalFilter(condition=[AND(=($1, $4), =($0, $3))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -298,10 +298,10 @@ LogicalProject(c=[$2], g=[$6]) Calc(select=[c, g]) +- SortMergeJoin(joinType=[InnerJoin], where=[((b = e) AND (a = d))], select=[a, b, c, d, e, g]) :- Exchange(distribution=[hash[b, a]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[e, d]]) +- Calc(select=[d, e, g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -313,8 +313,8 @@ Calc(select=[c, g]) @@ -323,9 +323,9 @@ Calc(select=[c, g]) +- SortMergeJoin(joinType=[InnerJoin], where=[((a = d) AND (b = e))], select=[d, e, g, a, b, c]) :- Exchange(distribution=[hash[d, e]]) : +- Calc(select=[d, e, g]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a, b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -338,8 +338,8 @@ Calc(select=[c, g]) LogicalProject(a=[$0], d=[$3]) +- LogicalFilter(condition=[=(+($0, 1), $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -348,10 +348,10 @@ Calc(select=[a, d]) +- SortMergeJoin(joinType=[InnerJoin], where=[($f3 = d)], select=[a, $f3, d]) :- Exchange(distribution=[hash[$f3]]) : +- Calc(select=[a, (a + 1) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -393,10 +393,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[true], joinType=[left]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -408,13 +408,13 @@ Calc(select=[CAST(2 AS INTEGER) AS a, b, d, CAST(e AS BIGINT) AS e]) : +- Exchange(distribution=[hash[a]]) : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) : +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, e]]) +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e]) +- Exchange(distribution=[hash[d]]) +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -434,10 +434,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($0, 2), =($3, 1))], joinType=[left]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -446,11 +446,11 @@ MultipleInput(members=[\nSortMergeJoin(joinType=[LeftOuterJoin], where=[((a = d) :- Exchange(distribution=[hash[a]]) : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) : +- Calc(select=[a, b]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -462,8 +462,8 @@ MultipleInput(members=[\nSortMergeJoin(joinType=[LeftOuterJoin], where=[((a = d) @@ -472,10 +472,10 @@ Calc(select=[c, g]) +- SortMergeJoin(joinType=[LeftOuterJoin], where=[((a = d) AND (d < 2))], select=[d, g, a, c]) :- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, g]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -487,8 +487,8 @@ Calc(select=[c, g]) @@ -497,10 +497,10 @@ Calc(select=[c, g]) +- SortMergeJoin(joinType=[RightOuterJoin], where=[(b = e)], select=[b, c, e, g]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[b, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[e, g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -520,10 +520,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($2, 2), =($1, 1))], joinType=[inner]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -533,11 +533,11 @@ Calc(select=[a, 1 AS b, CAST(2 AS INTEGER) AS d, 1 AS e]) :- Exchange(distribution=[hash[a]]) : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) : +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -550,8 +550,8 @@ Calc(select=[a, 1 AS b, CAST(2 AS INTEGER) AS d, 1 AS e]) LogicalProject(d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(IS NULL($3), <($0, 12))]) +- LogicalJoin(condition=[=($0, $3)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -560,10 +560,10 @@ Calc(select=[null:INTEGER AS d, e, f], where=[d IS NULL]) +- SortMergeJoin(joinType=[LeftOuterJoin], where=[(a = d)], select=[a, d, e, f]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a], where=[(a < 12)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, e, f], where=[(d < 12)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -576,8 +576,8 @@ Calc(select=[null:INTEGER AS d, e, f], where=[d IS NULL]) LogicalProject(d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(<($3, 10), <($0, 12))]) +- LogicalJoin(condition=[=($0, $3)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -586,10 +586,10 @@ Calc(select=[d, e, f]) +- SortMergeJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d, e, f]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a], where=[(a < 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, e, f], where=[(d < 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -602,8 +602,8 @@ Calc(select=[d, e, f]) LogicalProject(d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[=($3, null)]) +- LogicalJoin(condition=[=($0, $3)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -629,10 +629,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[true], joinType=[right]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -644,13 +644,13 @@ Calc(select=[a, CAST(b AS BIGINT) AS b, CAST(2 AS INTEGER) AS d, e]) : +- Exchange(distribution=[hash[a]]) : +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) : +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, e]]) +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e]) +- Exchange(distribution=[hash[d]]) +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -670,10 +670,10 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3]) +- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($2, 2), =($1, 1))], joinType=[right]) :- LogicalAggregate(group=[{0}], b=[COUNT($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0}], e=[COUNT($1)]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -682,11 +682,11 @@ MultipleInput(members=[\nSortMergeJoin(joinType=[RightOuterJoin], where=[((a = d :- Exchange(distribution=[hash[d]]) : +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0]) : +- Calc(select=[d, e]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -698,8 +698,8 @@ MultipleInput(members=[\nSortMergeJoin(joinType=[RightOuterJoin], where=[((a = d @@ -708,10 +708,10 @@ Calc(select=[c, g]) +- SortMergeJoin(joinType=[LeftOuterJoin], where=[(b = e)], select=[b, c, e, g]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[b, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[e, g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) ]]> @@ -723,8 +723,8 @@ Calc(select=[c, g]) @@ -732,9 +732,9 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7]) SortMergeJoin(joinType=[RightOuterJoin], where=[((a = d) AND (b < h))], select=[d, e, f, g, h, a, b, c]) :- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, e, f, g, h], where=[(d < 2)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -746,8 +746,8 @@ SortMergeJoin(joinType=[RightOuterJoin], where=[((a = d) AND (b < h))], select=[ @@ -756,10 +756,10 @@ Calc(select=[c, g]) +- SortMergeJoin(joinType=[RightOuterJoin], where=[(a = d)], select=[d, g, a, c]) :- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, g], where=[(d < 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[d, e, f, g, h]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) ]]> @@ -778,16 +778,16 @@ LogicalProject(k=[$0], v=[$1], k0=[$2], v0=[$3]) +- LogicalJoin(condition=[AND(=($0, $2), >($2, 10))], joinType=[left]) :- LogicalProject(k=[$0], v=[$1]) : +- LogicalFilter(condition=[=($0, 0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, src]]) +- LogicalProject(k=[$0], v=[$1]) +- LogicalFilter(condition=[=($0, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, src]]) ]]> @@ -799,14 +799,14 @@ Calc(select=[CAST(0 AS BIGINT) AS k, v, null:BIGINT AS k0, null:VARCHAR(21474836 @@ -818,14 +818,14 @@ Calc(select=[null:INTEGER AS d, null:BIGINT AS e, null:INTEGER AS f, null:VARCHA diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml index da4819342a7d6..fd85c5db67a15 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml @@ -24,12 +24,12 @@ limitations under the License. @@ -37,17 +37,17 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) SortMergeJoin(joinType=[LeftAntiJoin], where=[(a = i)], select=[a, b, c]) :- Exchange(distribution=[hash[a]]) : +- NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true]) -: :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: :- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[broadcast]) : +- Calc(select=[m IS NOT NULL AS $f0]) : +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) : +- Calc(select=[true AS i]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[i]]) +- Calc(select=[i], where=[(j < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -62,23 +62,23 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$0=[MAX($4)]) LogicalProject(d=[$0], $f1=[true], f=[$2], $f3=[1], e=[$1]) LogicalFilter(condition=[AND(=($cor0.b, $1), <($0, 100), =($cor0.c, $2))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -92,21 +92,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=($cor1.a, $0)]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })], variablesSet=[[$cor1]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -121,23 +121,23 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalAggregate(group=[{0}]) LogicalProject(e=[$1]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -150,19 +150,19 @@ SortMergeJoin(joinType=[LeftSemiJoin], where=[(c = f)], select=[a, b, c]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -174,9 +174,9 @@ SortMergeJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c]) ($1, 10), NOT(OR(LIKE($2, _UTF-16LE'abc'), NOT(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) +LogicalTableScan(table=[[default_catalog, default_database, r]]) })))))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -184,14 +184,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl Calc(select=[(a + 10) AS EXPR$0, c]) +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, c], build=[right], singleRowJoin=[true]) :- Calc(select=[a, c], where=[((b > 10) AND NOT(LIKE(c, 'abc')))]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[], where=[m IS NOT NULL]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -204,19 +204,19 @@ Calc(select=[(a + 10) AS EXPR$0, c]) LogicalProject(c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalJoin(condition=[=($1, $3)], joinType=[full]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -242,19 +242,19 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[c], build=[right], LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) +LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -283,22 +283,22 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], build=[r @@ -310,23 +310,23 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], build=[r ($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[], where=[m IS NOT NULL]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -341,19 +341,19 @@ LogicalProject(a=[$0]) LogicalUnion(all=[false]) LogicalProject(e=[$1], f=[$2]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1], k=[$2]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Calc(select=[j, k], where=[(i < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -380,13 +380,13 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a], build=[right], LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -395,13 +395,13 @@ SortMergeJoin(joinType=[LeftSemiJoin], where=[((a = i) AND (b = j))], select=[a, :- Exchange(distribution=[hash[a, b]]) : +- SortMergeJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c]) : :- Exchange(distribution=[hash[a]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[i, j]]) +- Calc(select=[i, j]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -414,12 +414,12 @@ SortMergeJoin(joinType=[LeftSemiJoin], where=[((a = i) AND (b = j))], select=[a, LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -428,13 +428,13 @@ SortMergeJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c]) :- Exchange(distribution=[hash[a]]) : +- SortMergeJoin(joinType=[LeftSemiJoin], where=[(b = j)], select=[a, b, c]) : :- Exchange(distribution=[hash[b]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[j]]) : +- Calc(select=[j]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -450,23 +450,23 @@ LogicalProject(EXPR$0=[$1]) LogicalAggregate(group=[{0}], EXPR$0=[MAX($1)]) LogicalProject(f=[$2], e=[$1]) LogicalFilter(condition=[AND(=($cor0.c, $2), <($0, 3))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -482,22 +482,22 @@ LogicalProject(EXPR$0=[$4], d=[$0]) LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$0=[COUNT()]) LogicalProject(d=[$0], $f1=[true], e=[$1], $f3=[1]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -511,9 +511,9 @@ LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) +- LogicalFilter(condition=[IN(SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=(+($cor0.b, 1), $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -522,10 +522,10 @@ Calc(select=[(a + 10) AS EXPR$0, c]) +- SortMergeJoin(joinType=[LeftSemiJoin], where=[(($f3 = EXPR$0) AND ($f4 = e))], select=[a, c, $f3, $f4]) :- Exchange(distribution=[hash[$f3, $f4]]) : +- Calc(select=[a, c, SUBSTRING(c, 1, 5) AS $f3, (b + 1) AS $f4]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[EXPR$0, e]]) +- Calc(select=[SUBSTRING(f, 1, 5) AS EXPR$0, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -539,9 +539,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), NOT(LIKE($2, _UTF-16LE'abc')), IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -549,10 +549,10 @@ LogicalProject(d=[$0]) SortMergeJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (b = e))], select=[a, b, c]) :- Exchange(distribution=[hash[a, b]]) : +- Calc(select=[a, b, c], where=[((b > 10) AND NOT(LIKE(c, 'abc')))]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, e]]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -568,12 +568,12 @@ LogicalProject(e=[$0]) LogicalJoin(condition=[=($0, $1)], joinType=[inner]) LogicalProject(e=[$1]) LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1]) LogicalFilter(condition=[AND(=($cor0.c, $2), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -581,16 +581,16 @@ LogicalProject(e=[$0]) Calc(select=[c]) +- SortMergeJoin(joinType=[LeftSemiJoin], where=[((b = e) AND (a = d) AND (c = k))], select=[a, b, c]) :- Exchange(distribution=[hash[b, a, c]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e, d, k]]) +- Calc(select=[e, d, k]) +- SortMergeJoin(joinType=[InnerJoin], where=[(e = j)], select=[e, d, j, k]) :- Exchange(distribution=[hash[e]]) : +- Calc(select=[e, d], where=[(e < 50)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j]]) +- Calc(select=[j, k], where=[(i > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -606,10 +606,10 @@ LogicalProject(j=[$3]) LogicalJoin(condition=[=($1, $4)], joinType=[left]) LogicalProject(e=[$1], f=[$2]) LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -617,16 +617,16 @@ LogicalProject(j=[$3]) Calc(select=[c]) +- SortMergeJoin(joinType=[LeftSemiJoin], where=[((b = j) AND (a = d))], select=[a, b, c]) :- Exchange(distribution=[hash[b, a]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[j, d]]) +- Calc(select=[j, d]) +- SortMergeJoin(joinType=[LeftOuterJoin], where=[(f = k)], select=[f, d, j, k]) :- Exchange(distribution=[hash[f]]) : +- Calc(select=[f, d], where=[(e < 50)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[k]]) +- Calc(select=[j, k]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -640,12 +640,12 @@ LogicalProject(c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalJoin(condition=[=($0, $3)], joinType=[right]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0]) LogicalFilter(condition=[AND(=($cor0.c, $2), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -654,16 +654,16 @@ Calc(select=[c]) +- SortMergeJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (c = k))], select=[a, c]) :- Exchange(distribution=[hash[a, c]]) : +- Calc(select=[a, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, k]]) +- Calc(select=[d, k]) +- SortMergeJoin(joinType=[RightOuterJoin], where=[(d = i)], select=[d, i, k]) :- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[i]]) +- Calc(select=[i, k], where=[(i > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -678,21 +678,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(f1=[$3]) LogicalFilter(condition=[=($cor1.a, $0)]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })], variablesSet=[[$cor1]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -706,9 +706,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, SUBSTRING($2, 1, 5), { LogicalProject(d=[$0], EXPR$1=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -717,10 +717,10 @@ Calc(select=[a, b, c]) +- SortMergeJoin(joinType=[LeftSemiJoin], where=[((a = d) AND ($f3 = EXPR$1) AND (b = e))], select=[a, b, c, $f3]) :- Exchange(distribution=[hash[a, $f3, b]]) : +- Calc(select=[a, b, c, SUBSTRING(c, 1, 5) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, EXPR$1, e]]) +- Calc(select=[d, SUBSTRING(f, 1, 5) AS EXPR$1, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -734,19 +734,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> e))], select=[a, b, c]) :- Exchange(distribution=[hash[a]]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -760,9 +760,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($cor0.b, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -770,10 +770,10 @@ LogicalProject(d=[$0]) SortMergeJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[(b > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -788,16 +788,16 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(rk=[$2]) LogicalFilter(condition=[<>($cor0.a, $0)]) LogicalProject(d=[$0], e=[$1], rk=[RANK() OVER (PARTITION BY $0 ORDER BY $1 NULLS FIRST)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> d))], select=[a, b, c]) :- Exchange(distribution=[hash[b]]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[rk]]) +- Calc(select=[w0$o0 AS rk, d]) +- OverAggregate(partitionBy=[d], orderBy=[e ASC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[d, e, w0$o0]) @@ -805,7 +805,7 @@ SortMergeJoin(joinType=[LeftSemiJoin], where=[((b = rk) AND (a <> d))], select=[ +- Sort(orderBy=[d ASC, e ASC]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -821,15 +821,15 @@ LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) LogicalProject(d=[$0]) LogicalFilter(condition=[IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), { LogicalProject(i=[$0]) LogicalFilter(condition=[=($2, $cor0.c)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -839,7 +839,7 @@ Calc(select=[a]) :- Exchange(distribution=[hash[EXPR$0, c]]) : +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[a, c, EXPR$0], build=[right], singleRowJoin=[true]) : :- Calc(select=[a, c]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0]) : +- Exchange(distribution=[single]) @@ -848,11 +848,11 @@ Calc(select=[a]) : +- MultipleInput(members=[\nSortMergeJoin(joinType=[InnerJoin], where=[(e = j)], select=[d, e, j])\n:- [#1] Exchange(distribution=[hash[e]])\n+- HashAggregate(isMerge=[true], groupBy=[j], select=[j])\n +- [#2] Exchange(distribution=[hash[j]])\n]) : :- Exchange(distribution=[hash[e]]) : : +- Calc(select=[d, e]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) : +- Exchange(distribution=[hash[j]]) : +- LocalHashAggregate(groupBy=[j], select=[j]) : +- Calc(select=[j]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) +- Exchange(distribution=[hash[i, k]]) +- Calc(select=[i, k]) +- Reused(reference_id=[1]) @@ -869,19 +869,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -895,9 +895,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 1), IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -905,10 +905,10 @@ LogicalProject(d=[$0]) SortMergeJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (b = e))], select=[a, b, c]) :- Exchange(distribution=[hash[a, b]]) : +- Calc(select=[a, b, c], where=[(b > 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, e]]) +- Calc(select=[d, e], where=[(d > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -922,9 +922,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=(CAST($cor0.b):INTEGER, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -932,10 +932,10 @@ LogicalProject(d=[$0]) SortMergeJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[(a = CAST(b AS INTEGER))]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -951,9 +951,9 @@ LogicalProject(EXPR$0=[$1]) LogicalAggregate(group=[{0}], EXPR$0=[MAX($1)]) LogicalProject(f=[$2], e=[$1]) LogicalFilter(condition=[<($0, 3)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -962,14 +962,14 @@ Calc(select=[a]) +- SortMergeJoin(joinType=[LeftSemiJoin], where=[(b = EXPR$0)], select=[a, b]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[EXPR$0]]) +- Calc(select=[EXPR$0]) +- HashAggregate(isMerge=[true], groupBy=[f], select=[f, Final_MAX(max$0) AS EXPR$0]) +- Exchange(distribution=[hash[f]]) +- LocalHashAggregate(groupBy=[f], select=[f, Partial_MAX(e) AS max$0]) +- Calc(select=[f, e], where=[(d < 3)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -983,22 +983,22 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, $1, { LogicalAggregate(group=[{0}], EXPR$1=[COUNT()]) LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1011,9 +1011,9 @@ SortMergeJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (b = EXPR$1))], selec LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), NOT(LIKE($2, _UTF-16LE'abc')), IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1021,10 +1021,10 @@ LogicalProject(d=[$0]) SortMergeJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[((b > 10) AND NOT(LIKE(c, 'abc')))]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1037,19 +1037,19 @@ SortMergeJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IS TRUE(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1064,12 +1064,12 @@ LogicalProject(a=[$0]) LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) LogicalProject(e=[$1]) LogicalFilter(condition=[>($0, 0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1079,16 +1079,16 @@ Calc(select=[a]) :- Exchange(distribution=[hash[EXPR$0]]) : +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[a, EXPR$0], build=[right], singleRowJoin=[true]) : :- Calc(select=[a]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_MAX(e) AS max$0]) : +- Calc(select=[e], where=[(d > 0)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j]]) +- Calc(select=[j]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1102,15 +1102,15 @@ LogicalProject(b=[$1]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($1, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), >($1, $SCALAR_QUERY({ LogicalProject(EXPR$0=[*(0.5:DECIMAL(2, 1), $0)]) LogicalAggregate(group=[{}], agg#0=[SUM($0)]) LogicalProject(j=[$1]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1124,13 +1124,13 @@ Calc(select=[b]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_SUM(j) AS sum$0]) : +- Calc(select=[j], where=[(i < 100)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d], where=[(e > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1143,11 +1143,11 @@ Calc(select=[b]) LogicalProject(s=[$1]) +- LogicalFilter(condition=[AND(>($2, 2), IN($3, { LogicalProject(e=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) +- LogicalAggregate(group=[{0}], s=[SUM($1)], agg#1=[COUNT()], agg#2=[MAX($0)]) +- LogicalProject(b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1160,10 +1160,10 @@ Calc(select=[s]) : +- Exchange(distribution=[hash[b]]) : +- LocalHashAggregate(groupBy=[b], select=[b, Partial_SUM(a) AS sum$0, Partial_COUNT(*) AS count1$1, Partial_MAX(b) AS max$2]) : +- Calc(select=[b, a]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1181,13 +1181,13 @@ LogicalProject(c=[$2]) LogicalProject(d=[$0]) LogicalProject(d=[$0], e=[$1], f=[$2], i=[$3]) LogicalJoin(condition=[=($1, $4)], joinType=[full]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0], i0=[CAST($0):BIGINT]) LogicalProject(i=[$0]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1196,16 +1196,16 @@ Calc(select=[c]) +- SortMergeJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, c]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) +- SortMergeJoin(joinType=[FullOuterJoin], where=[(e = i0)], select=[d, e, i0]) :- Exchange(distribution=[hash[e]]) : +- Calc(select=[d, e]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[i0]]) +- Calc(select=[CAST(i AS BIGINT) AS i0], where=[(i > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1219,21 +1219,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($2, { LogicalProject(f1=[$3]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1249,9 +1249,9 @@ SELECT * FROM l WHERE LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN(+($0, 10), SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[+($0, 100)], EXPR$1=[SUBSTRING($2, 1, 5)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1260,10 +1260,10 @@ Calc(select=[a, b, c]) +- SortMergeJoin(joinType=[LeftSemiJoin], where=[(($f3 = EXPR$0) AND ($f4 = EXPR$1))], select=[a, b, c, $f3, $f4]) :- Exchange(distribution=[hash[$f3, $f4]]) : +- Calc(select=[a, b, c, (a + 10) AS $f3, SUBSTRING(c, 1, 5) AS $f4]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[EXPR$0, EXPR$1]]) +- Calc(select=[(d + 100) AS EXPR$0, SUBSTRING(f, 1, 5) AS EXPR$1]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1279,16 +1279,16 @@ SELECT * FROM l WHERE (a, b) IN LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, $1, { LogicalProject(EXPR$0=[MAX($0) OVER ()], EXPR$1=[MIN($1) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1310,19 +1310,19 @@ SortMergeJoin(joinType=[LeftSemiJoin], where=[((a = $0) AND (b = $1))], select=[ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1336,9 +1336,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), >($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1346,10 +1346,10 @@ LogicalProject(d=[$0]) SortMergeJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[(b > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d], where=[(e < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1362,9 +1362,9 @@ SortMergeJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN(+($0, 1), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1373,10 +1373,10 @@ Calc(select=[a, b, c]) +- SortMergeJoin(joinType=[LeftSemiJoin], where=[($f3 = d)], select=[a, b, c, $f3]) :- Exchange(distribution=[hash[$f3]]) : +- Calc(select=[a, b, c, (a + 1) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1394,12 +1394,12 @@ LogicalProject(a=[$0]) LogicalUnion(all=[false]) LogicalProject(e=[$1]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1408,14 +1408,14 @@ Calc(select=[a]) +- MultipleInput(members=[\nSortMergeJoin(joinType=[LeftSemiJoin], where=[(b = e)], select=[a, b])\n:- [#1] Exchange(distribution=[hash[b]])\n+- HashAggregate(isMerge=[true], groupBy=[e], select=[e])\n +- [#2] Exchange(distribution=[hash[e]])\n]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- LocalHashAggregate(groupBy=[e], select=[e]) +- Union(all=[true], union=[e]) :- Calc(select=[e], where=[(d > 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Calc(select=[CAST(i AS BIGINT) AS i], where=[(i < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1428,29 +1428,29 @@ Calc(select=[a]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[AND(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) +LogicalTableScan(table=[[default_catalog, default_database, t]]) }), =($cor0.a, $0))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1463,14 +1463,14 @@ SortMergeJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c]) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), EXISTS({ LogicalFilter(condition=[>($0, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), >=($1, 1), EXISTS({ LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1482,7 +1482,7 @@ MultipleInput(readOrder=[0,0,1,1], members=[\nNestedLoopJoin(joinType=[LeftSemiJ : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) : +- Calc(select=[true AS i], where=[(j < 100)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[true AS $f0], where=[m IS NOT NULL]) : +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) @@ -1492,9 +1492,9 @@ MultipleInput(readOrder=[0,0,1,1], members=[\nNestedLoopJoin(joinType=[LeftSemiJ : +- Reused(reference_id=[1]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[(b >= 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1507,14 +1507,14 @@ MultipleInput(readOrder=[0,0,1,1], members=[\nNestedLoopJoin(joinType=[LeftSemiJ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), NOT(EXISTS({ LogicalFilter(condition=[AND(=($cor0.b, $1), >(CAST($2):BIGINT, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), >=(CAST($2):BIGINT, 1), NOT(EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1526,12 +1526,12 @@ SortMergeJoin(joinType=[LeftAntiJoin], where=[(a = i)], select=[a, b, c, d, e, f : : +- SortMergeJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f]) : : :- Exchange(distribution=[hash[a]]) : : : +- Calc(select=[a, b, c], where=[(CAST(c AS BIGINT) >= 1)]) -: : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : : +- Exchange(distribution=[hash[d]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) : +- Exchange(distribution=[hash[j]]) : +- Calc(select=[j], where=[(CAST(k AS BIGINT) > 50)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) +- Exchange(distribution=[hash[i]]) +- Calc(select=[i], where=[(j < 100)]) +- Reused(reference_id=[1]) @@ -1548,13 +1548,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) LogicalFilter(condition=[AND(=($cor0.a, $0), <>($2, _UTF-16LE'test'))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1563,13 +1563,13 @@ SortMergeJoin(joinType=[LeftSemiJoin], where=[((b = j) AND (a = i))], select=[a, :- Exchange(distribution=[hash[b, a]]) : +- SortMergeJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (c = f))], select=[a, b, c]) : :- Exchange(distribution=[hash[a, c]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d, f]]) : +- Calc(select=[d, f]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j, i]]) +- Calc(select=[j, i], where=[(k <> 'test')]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1585,26 +1585,26 @@ LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), IN($2, { LogicalProject(k=[$2]) LogicalFilter(condition=[=($cor1.e, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor1]]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1617,13 +1617,13 @@ SortMergeJoin(joinType=[LeftSemiJoin], where=[((a = d) AND (b = e))], select=[a, LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) LogicalFilter(condition=[=($2, $cor0.c)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1632,13 +1632,13 @@ SortMergeJoin(joinType=[LeftSemiJoin], where=[((b = j) AND (k = c))], select=[a, :- Exchange(distribution=[hash[b, c]]) : +- SortMergeJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c]) : :- Exchange(distribution=[hash[a]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j, k]]) +- Calc(select=[j, k]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1651,12 +1651,12 @@ SortMergeJoin(joinType=[LeftSemiJoin], where=[((b = j) AND (k = c))], select=[a, LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1665,13 +1665,13 @@ SortMergeJoin(joinType=[LeftSemiJoin], where=[(b = j)], select=[a, b, c]) :- Exchange(distribution=[hash[b]]) : +- SortMergeJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c]) : :- Exchange(distribution=[hash[a]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j]]) +- Calc(select=[j]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1686,27 +1686,27 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(d=[$0]) LogicalFilter(condition=[IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1719,14 +1719,14 @@ SortMergeJoin(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c]) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), NOT(EXISTS({ LogicalFilter(condition=[>($0, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), >=($1, 1), NOT(EXISTS({ LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1738,7 +1738,7 @@ MultipleInput(readOrder=[0,0,1,1], members=[\nNestedLoopJoin(joinType=[LeftAntiJ : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) : +- Calc(select=[true AS i], where=[(j < 100)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[m IS NOT NULL AS $f0]) : +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) @@ -1748,9 +1748,9 @@ MultipleInput(readOrder=[0,0,1,1], members=[\nNestedLoopJoin(joinType=[LeftAntiJ : +- Reused(reference_id=[1]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[(b >= 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1763,19 +1763,19 @@ MultipleInput(readOrder=[0,0,1,1], members=[\nNestedLoopJoin(joinType=[LeftAntiJ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1787,9 +1787,9 @@ SortMergeJoin(joinType=[LeftAntiJoin], where=[(a = d)], select=[a, b, c]) ($1, 10), LIKE($2, _UTF-16LE'abc'), NOT(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) +LogicalTableScan(table=[[default_catalog, default_database, r]]) }))))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1797,14 +1797,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl Calc(select=[(a + 10) AS EXPR$0, c]) +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, c], build=[right], singleRowJoin=[true]) :- Calc(select=[a, c], where=[((b <= 10) AND NOT(LIKE(c, 'abc')))]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[], where=[m IS NOT NULL]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1816,22 +1816,22 @@ Calc(select=[(a + 10) AS EXPR$0, c]) @@ -1843,23 +1843,23 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c], build=[ri ($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- Calc(select=[m IS NOT NULL AS $f0]) +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1874,20 +1874,20 @@ LogicalProject(a=[$0], b=[$1]) LogicalFilter(condition=[=($cor0.a, $0)]) LogicalAggregate(group=[{0}]) LogicalProject(c=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, rightT]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, leftT]]) ]]> @@ -1900,13 +1900,13 @@ MultipleInput(members=[\nSortMergeJoin(joinType=[LeftAntiJoin], where=[(a = c)], LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })), NOT(IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1915,13 +1915,13 @@ SortMergeJoin(joinType=[LeftAntiJoin], where=[((a IS NULL OR i IS NULL OR (a = i :- Exchange(distribution=[hash[b]]) : +- SortMergeJoin(joinType=[LeftAntiJoin], where=[(a = d)], select=[a, b, c]) : :- Exchange(distribution=[hash[a]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j]]) +- Calc(select=[i, j]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1935,9 +1935,9 @@ LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) +- LogicalFilter(condition=[NOT(IN(SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=(+($cor0.b, 1), $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1946,10 +1946,10 @@ Calc(select=[(a + 10) AS EXPR$0, c]) +- SortMergeJoin(joinType=[LeftAntiJoin], where=[(($f3 IS NULL OR EXPR$0 IS NULL OR ($f3 = EXPR$0)) AND ($f4 = e))], select=[a, c, $f3, $f4]) :- Exchange(distribution=[hash[$f4]]) : +- Calc(select=[a, c, SUBSTRING(c, 1, 5) AS $f3, (b + 1) AS $f4]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[SUBSTRING(f, 1, 5) AS EXPR$0, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1962,16 +1962,16 @@ Calc(select=[(a + 10) AS EXPR$0, c]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[NOT(IN(CASE(NOT(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), 1, NOT(IN($0, { LogicalProject(EXPR$0=[CAST($1):INTEGER]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), 2, 3), { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1986,12 +1986,12 @@ Calc(select=[b]) : : +- Exchange(distribution=[single]) : : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1]) : : +- Calc(select=[CAST(j AS INTEGER) AS EXPR$0])(reuse_id=[1]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[2]) + : : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[2]) : :- Exchange(distribution=[hash[EXPR$0]]) : : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) : : +- Reused(reference_id=[1]) : :- Exchange(distribution=[hash[a]]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : :- Exchange(distribution=[broadcast]) : : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : : +- Exchange(distribution=[single]) @@ -2003,7 +2003,7 @@ Calc(select=[b]) : +- Reused(reference_id=[3]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[d, f]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2017,19 +2017,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, $2, { LogicalProject(d=[$0], f=[$2]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2043,19 +2043,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2069,9 +2069,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 1), NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2079,10 +2079,10 @@ LogicalProject(d=[$0]) SortMergeJoin(joinType=[LeftAntiJoin], where=[((a IS NULL OR d IS NULL OR (a = d)) AND (b = e))], select=[a, b, c]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[a, b, c], where=[(b > 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[d, e], where=[(d > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/AggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/AggregateTest.xml index 31465c74770d0..6f6ce9785d010 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/AggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/AggregateTest.xml @@ -21,7 +21,7 @@ limitations under the License. @@ -29,7 +29,7 @@ LogicalProject(EXPR$0=[$0], EXPR$1=[$1], EXPR$2=[$2]) HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final_SUM(sum$2) AS EXPR$1, Final_COUNT(count$3) AS EXPR$2]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -40,7 +40,7 @@ LogicalProject(EXPR$0=[$0], EXPR$1=[$1], EXPR$2=[$2]) +- LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($2)]) +- LogicalFilter(condition=[=($0, 1)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -49,7 +49,7 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3]) +- Calc(select=[CAST(1 AS INTEGER) AS a, b, c], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -61,7 +61,7 @@ LogicalProject(EXPR$0=[$0], EXPR$1=[$1], EXPR$2=[$2], EXPR$3=[$3]) +- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[$2._1]) +- LogicalFilter(condition=[=($0, 1)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -70,7 +70,7 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3, Partial_SUM($f3) AS sum$4]) +- Calc(select=[CAST(1 AS INTEGER) AS a, b, c, c._1 AS $f3], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -80,7 +80,7 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final LogicalFilter(condition=[=($0, 1)]) +- LogicalProject(a=[$0], EXPR$0=[$1], EXPR$1=[$2], EXPR$2=[$3]) +- LogicalAggregate(group=[{0}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -90,7 +90,7 @@ Calc(select=[CAST(1 AS INTEGER) AS a, EXPR$0, EXPR$1, EXPR$2]) +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a, Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3]) +- Calc(select=[a, b, c], where=[(a = 1)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CalcTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CalcTest.xml index 1731a069a085b..25e0154316c39 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CalcTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CalcTest.xml @@ -20,13 +20,13 @@ limitations under the License. @@ -37,13 +37,13 @@ LogicalFilter(condition=[=(MOD($0, 2), 1)]) +- LogicalFilter(condition=[<($1, 2)]) +- LogicalFilter(condition=[>($0, 0)]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 0) AND (b < 2) AND (MOD(a, 2) = 1))]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -51,13 +51,13 @@ Calc(select=[a, b], where=[((a > 0) AND (b < 2) AND (MOD(a, 2) = 1))]) @@ -65,13 +65,13 @@ Calc(select=[a._1 AS a$_1, a._2 AS a$_2, c, b._1 AS b$_1, b._2 AS b$_2]) @@ -80,7 +80,7 @@ Calc(select=[a._1 AS a$_1, a._2 AS a$_2, b]) @@ -88,7 +88,7 @@ LogicalProject(EXPR$0=[$0], EXPR$1=[$1]) HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS EXPR$0, Final_MAX(max$1) AS EXPR$1]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_SUM(a) AS sum$0, Partial_MAX(b) AS max$1]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -97,7 +97,7 @@ HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS EXPR$0, Final_MAX(max$ @@ -106,7 +106,7 @@ Calc(select=[EXPR$0]) +- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_SUM(sum$0) AS EXPR$0]) +- Exchange(distribution=[hash[c]]) +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(a) AS sum$0]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -115,7 +115,7 @@ Calc(select=[EXPR$0]) @@ -123,7 +123,7 @@ LogicalProject(a=[$0], c=[$1]) HashAggregate(isMerge=[true], groupBy=[a, c], select=[a, c]) +- Exchange(distribution=[hash[a, c]]) +- LocalHashAggregate(groupBy=[a, c], select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -133,7 +133,7 @@ HashAggregate(isMerge=[true], groupBy=[a, c], select=[a, c]) LogicalFilter(condition=[=($1, 2)]) +- LogicalProject(word=[$0], frequency=[$1]) +- LogicalAggregate(group=[{0}], EXPR$0=[SUM($1)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(word, frequency)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -142,7 +142,7 @@ Calc(select=[word, CAST(2 AS BIGINT) AS frequency], where=[(EXPR$0 = 2)]) +- HashAggregate(isMerge=[true], groupBy=[word], select=[word, Final_SUM(sum$0) AS EXPR$0]) +- Exchange(distribution=[hash[word]]) +- LocalHashAggregate(groupBy=[word], select=[word, Partial_SUM(frequency) AS sum$0]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(word, frequency)]]], fields=[word, frequency]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[word, frequency]) ]]> @@ -151,7 +151,7 @@ Calc(select=[word, CAST(2 AS BIGINT) AS frequency], where=[(EXPR$0 = 2)]) @@ -160,7 +160,7 @@ Calc(select=[a]) +- HashAggregate(isMerge=[true], groupBy=[a, c], select=[a, c]) +- Exchange(distribution=[hash[a, c]]) +- LocalHashAggregate(groupBy=[a, c], select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -170,7 +170,7 @@ Calc(select=[a]) LogicalProject(EXPR$0=[$1]) +- LogicalAggregate(group=[{1}], EXPR$0=[SUM($0)]) +- LogicalProject(a=[$0], k=[*org.apache.flink.table.planner.plan.batch.table.CalcTest$MyHashCode$*($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -180,7 +180,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[hash[k]]) +- LocalHashAggregate(groupBy=[k], select=[k, Partial_SUM(a) AS sum$0]) +- Calc(select=[a, *org.apache.flink.table.planner.plan.batch.table.CalcTest$MyHashCode$*(c) AS k]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -190,7 +190,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$1]) +- LogicalAggregate(group=[{1}], EXPR$0=[SUM($0)]) +- LogicalProject(a=[$0], k=[UPPER($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -200,7 +200,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[hash[k]]) +- LocalHashAggregate(groupBy=[k], select=[k, Partial_SUM(a) AS sum$0]) +- Calc(select=[a, UPPER(c) AS k]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -208,13 +208,13 @@ Calc(select=[EXPR$0]) @@ -222,13 +222,13 @@ Calc(select=[a, b]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/ColumnFunctionsTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/ColumnFunctionsTest.xml index 9d55917686728..770653e7af7ae 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/ColumnFunctionsTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/ColumnFunctionsTest.xml @@ -20,14 +20,14 @@ limitations under the License. diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.xml index a2d90a00ed11a..82f48b4ba8844 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.xml @@ -25,7 +25,7 @@ LogicalProject(c=[$0], d=[$1]) +- LogicalProject(c=[$2], d=[$3], e=[$4]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)]) ]]> @@ -33,7 +33,7 @@ LogicalProject(c=[$0], d=[$1]) ($1, 20)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -41,14 +41,14 @@ Calc(select=[c, d]) @@ -61,7 +61,7 @@ LogicalProject(c=[$0], d=[$1]) +- LogicalProject(c=[$2], d=[$3], e=[$4]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)]) ]]> @@ -69,7 +69,7 @@ LogicalProject(c=[$0], d=[$1]) ($1, 20)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -78,7 +78,7 @@ Calc(select=[c, d]) @@ -86,7 +86,7 @@ LogicalProject(c=[$2], s=[$3]) @@ -95,7 +95,7 @@ Calc(select=[c, s]) @@ -103,7 +103,7 @@ LogicalProject(c=[$2], s=[$3]) @@ -113,7 +113,7 @@ Calc(select=[c, s]) LogicalFilter(condition=[>($1, _UTF-16LE'')]) +- LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{}]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> @@ -121,7 +121,7 @@ LogicalFilter(condition=[>($1, _UTF-16LE'')]) '')]) +- Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], correlate=[table(*org.apache.flink.table.planner.utils.TableFunc1*(c))], select=[a,b,c,s], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)], joinType=[LEFT]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -130,7 +130,7 @@ Calc(select=[c, s], where=[(s > '')]) @@ -138,7 +138,7 @@ LogicalProject(c=[$2], s=[$3]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/GroupWindowTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/GroupWindowTest.xml index 5326122dc42b8..f34a76741c48a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/GroupWindowTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/GroupWindowTest.xml @@ -21,7 +21,7 @@ limitations under the License. @@ -29,7 +29,7 @@ LogicalProject(EXPR$0=[$0]) HashWindowAggregate(window=[TumblingGroupWindow('w, long, 5)], select=[Final_COUNT(count$0) AS EXPR$0]) +- Exchange(distribution=[single]) +- LocalHashWindowAggregate(window=[TumblingGroupWindow('w, long, 5)], select=[Partial_COUNT(int) AS count$0]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(long, int, string)]]], fields=[long, int, string]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[long, int, string]) ]]> @@ -38,7 +38,7 @@ HashWindowAggregate(window=[TumblingGroupWindow('w, long, 5)], select=[Final_COU @@ -46,7 +46,7 @@ LogicalProject(string=[$0], EXPR$0=[$1]) HashWindowAggregate(groupBy=[string], window=[SlidingGroupWindow('w, long, 8, 10)], select=[string, Final_COUNT(count$0) AS EXPR$0]) +- Exchange(distribution=[hash[string]]) +- LocalHashWindowAggregate(groupBy=[string], window=[SlidingGroupWindow('w, long, 8, 10)], select=[string, Partial_COUNT(int) AS count$0]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(long, int, string)]]], fields=[long, int, string]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[long, int, string]) ]]> @@ -55,7 +55,7 @@ HashWindowAggregate(groupBy=[string], window=[SlidingGroupWindow('w, long, 8, 10 @@ -63,7 +63,7 @@ LogicalProject(string=[$0], EXPR$0=[$1]) HashWindowAggregate(groupBy=[string], window=[TumblingGroupWindow('w, long, 5)], select=[string, Final_COUNT(count$0) AS EXPR$0]) +- Exchange(distribution=[hash[string]]) +- LocalHashWindowAggregate(groupBy=[string], window=[TumblingGroupWindow('w, long, 5)], select=[string, Partial_COUNT(int) AS count$0]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(long, int, string)]]], fields=[long, int, string]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[long, int, string]) ]]> @@ -72,7 +72,7 @@ HashWindowAggregate(groupBy=[string], window=[TumblingGroupWindow('w, long, 5)], @@ -80,7 +80,7 @@ LogicalProject(string=[$0], EXPR$0=[$1], EXPR$1=[$2], EXPR$2=[$3], EXPR$3=[$4]) HashWindowAggregate(groupBy=[string], window=[TumblingGroupWindow('w, ts, 7200000)], properties=[EXPR$1, EXPR$2, EXPR$3], select=[string, Final_COUNT(count$0) AS EXPR$0]) +- Exchange(distribution=[hash[string]]) +- LocalHashWindowAggregate(groupBy=[string], window=[TumblingGroupWindow('w, ts, 7200000)], properties=[EXPR$1, EXPR$2, EXPR$3], select=[string, Partial_COUNT(int) AS count$0]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(ts, int, string)]]], fields=[ts, int, string]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[ts, int, string]) ]]> @@ -90,7 +90,7 @@ HashWindowAggregate(groupBy=[string], window=[TumblingGroupWindow('w, ts, 720000 LogicalProject(string=[$0], EXPR$0=[$1]) +- LogicalWindowAggregate(group=[{2}], EXPR$0=[*org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions$WeightedAvgWithMerge*($0, $3)], window=[TumblingGroupWindow('w, long, 5)], properties=[]) +- LogicalProject(long=[$0], int=[$1], string=[$2], int0=[CAST($1):BIGINT]) - +- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(long, int, string)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Table1]]) ]]> @@ -102,7 +102,7 @@ SortWindowAggregate(groupBy=[string], window=[TumblingGroupWindow('w, long, 5)], +- LocalSortWindowAggregate(groupBy=[string], window=[TumblingGroupWindow('w, long, 5)], select=[string, Partial_*org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions$WeightedAvgWithMerge*(long, int0) AS EXPR$0]) +- Calc(select=[long, int, string, CAST(int AS BIGINT) AS int0]) +- Sort(orderBy=[string ASC, long ASC]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(long, int, string)]]], fields=[long, int, string]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[long, int, string]) ]]> @@ -111,7 +111,7 @@ SortWindowAggregate(groupBy=[string], window=[TumblingGroupWindow('w, long, 5)], @@ -119,7 +119,7 @@ LogicalProject(string=[$0], EXPR$0=[$1], EXPR$1=[$2], EXPR$2=[$3], EXPR$3=[$4]) HashWindowAggregate(groupBy=[string], window=[TumblingGroupWindow('w, ts, 7200000)], properties=[EXPR$1, EXPR$2, EXPR$3], select=[string, Final_COUNT(count$0) AS EXPR$0]) +- Exchange(distribution=[hash[string]]) +- LocalHashWindowAggregate(groupBy=[string], window=[TumblingGroupWindow('w, ts, 7200000)], properties=[EXPR$1, EXPR$2, EXPR$3], select=[string, Partial_COUNT(int) AS count$0]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(ts, int, string)]]], fields=[ts, int, string]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[ts, int, string]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/JoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/JoinTest.xml index ea214042a2ad9..2aa6f01e2ef61 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/JoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/JoinTest.xml @@ -22,8 +22,8 @@ limitations under the License. LogicalFilter(condition=[>=($0, 0)]) +- LogicalProject(c1=[*org.apache.flink.table.planner.plan.batch.table.JoinTest$Merger$*($2, *org.apache.flink.table.planner.plan.batch.table.JoinTest$Merger$*($2, $5))]) +- LogicalJoin(condition=[=($1, $4)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Table2, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, Table1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Table2]]) ]]> @@ -32,10 +32,10 @@ Calc(select=[*org.apache.flink.table.planner.plan.batch.table.JoinTest$Merger$*( +- HashJoin(joinType=[LeftOuterJoin], where=[(b = e)], select=[b, c, e, f], build=[left]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[b, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[e, f]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, Table2]], fields=[d, e, f]) ]]> @@ -44,8 +44,8 @@ Calc(select=[*org.apache.flink.table.planner.plan.batch.table.JoinTest$Merger$*( @@ -54,10 +54,10 @@ Calc(select=[b, y]) +- HashJoin(joinType=[FullOuterJoin], where=[((a = z) AND (b < 2))], select=[a, b, y, z], build=[left]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c]) +- Exchange(distribution=[hash[z]]) +- Calc(select=[y, z]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, S, source: [TestTableSource(x, y, z)]]], fields=[x, y, z]) + +- TableSourceScan(table=[[default_catalog, default_database, S]], fields=[x, y, z]) ]]> @@ -66,8 +66,8 @@ Calc(select=[b, y]) @@ -76,9 +76,9 @@ Calc(select=[b, y]) +- HashJoin(joinType=[FullOuterJoin], where=[((a = z) AND (b < x))], select=[a, b, x, y, z], build=[left]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c]) +- Exchange(distribution=[hash[z]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, S, source: [TestTableSource(x, y, z)]]], fields=[x, y, z]) + +- TableSourceScan(table=[[default_catalog, default_database, S]], fields=[x, y, z]) ]]> @@ -87,8 +87,8 @@ Calc(select=[b, y]) @@ -97,10 +97,10 @@ Calc(select=[b, y]) +- HashJoin(joinType=[FullOuterJoin], where=[(a = z)], select=[a, b, y, z], build=[left]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c]) +- Exchange(distribution=[hash[z]]) +- Calc(select=[y, z]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, S, source: [TestTableSource(x, y, z)]]], fields=[x, y, z]) + +- TableSourceScan(table=[[default_catalog, default_database, S]], fields=[x, y, z]) ]]> @@ -109,8 +109,8 @@ Calc(select=[b, y]) @@ -119,10 +119,10 @@ Calc(select=[b, y]) +- HashJoin(joinType=[LeftOuterJoin], where=[((a = z) AND (b < 2))], select=[a, b, y, z], build=[left]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c]) +- Exchange(distribution=[hash[z]]) +- Calc(select=[y, z]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, S, source: [TestTableSource(x, y, z)]]], fields=[x, y, z]) + +- TableSourceScan(table=[[default_catalog, default_database, S]], fields=[x, y, z]) ]]> @@ -131,8 +131,8 @@ Calc(select=[b, y]) @@ -141,9 +141,9 @@ Calc(select=[b, y]) +- HashJoin(joinType=[LeftOuterJoin], where=[((a = z) AND (b < x))], select=[a, b, x, y, z], build=[left]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c]) +- Exchange(distribution=[hash[z]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, S, source: [TestTableSource(x, y, z)]]], fields=[x, y, z]) + +- TableSourceScan(table=[[default_catalog, default_database, S]], fields=[x, y, z]) ]]> @@ -152,8 +152,8 @@ Calc(select=[b, y]) @@ -162,10 +162,10 @@ Calc(select=[b, y]) +- HashJoin(joinType=[LeftOuterJoin], where=[(a = z)], select=[a, b, y, z], build=[left]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c]) +- Exchange(distribution=[hash[z]]) +- Calc(select=[y, z]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, S, source: [TestTableSource(x, y, z)]]], fields=[x, y, z]) + +- TableSourceScan(table=[[default_catalog, default_database, S]], fields=[x, y, z]) ]]> @@ -175,18 +175,18 @@ Calc(select=[b, y]) LogicalProject(c=[$2], g=[$6]) +- LogicalFilter(condition=[=($3, $5)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, Table3, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Table5, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, Table3]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Table5]]) ]]> @@ -196,8 +196,8 @@ NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[c, g], build=[right]) LogicalProject(c=[$2], g=[$6]) +- LogicalFilter(condition=[<($0, $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, Table3, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Table5, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, Table3]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Table5]]) ]]> @@ -206,9 +206,9 @@ Calc(select=[c, g]) +- NestedLoopJoin(joinType=[InnerJoin], where=[(a < d)], select=[a, c, d, g], build=[left]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table3, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, Table3]], fields=[a, b, c]) +- Calc(select=[d, g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table5, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) + +- TableSourceScan(table=[[default_catalog, default_database, Table5]], fields=[d, e, f, g, h]) ]]> @@ -217,8 +217,8 @@ Calc(select=[c, g]) @@ -227,10 +227,10 @@ Calc(select=[b, x]) +- HashJoin(joinType=[RightOuterJoin], where=[((a = z) AND (x < 2))], select=[a, b, x, z], build=[left]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c]) +- Exchange(distribution=[hash[z]]) +- Calc(select=[x, z]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, S, source: [TestTableSource(x, y, z)]]], fields=[x, y, z]) + +- TableSourceScan(table=[[default_catalog, default_database, S]], fields=[x, y, z]) ]]> @@ -239,8 +239,8 @@ Calc(select=[b, x]) @@ -249,9 +249,9 @@ Calc(select=[b, y]) +- HashJoin(joinType=[RightOuterJoin], where=[((a = z) AND (b < x))], select=[a, b, x, y, z], build=[left]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c]) +- Exchange(distribution=[hash[z]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, S, source: [TestTableSource(x, y, z)]]], fields=[x, y, z]) + +- TableSourceScan(table=[[default_catalog, default_database, S]], fields=[x, y, z]) ]]> @@ -260,8 +260,8 @@ Calc(select=[b, y]) @@ -270,10 +270,10 @@ Calc(select=[b, y]) +- HashJoin(joinType=[RightOuterJoin], where=[(a = z)], select=[a, b, y, z], build=[left]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c]) +- Exchange(distribution=[hash[z]]) +- Calc(select=[y, z]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, S, source: [TestTableSource(x, y, z)]]], fields=[x, y, z]) + +- TableSourceScan(table=[[default_catalog, default_database, S]], fields=[x, y, z]) ]]> @@ -281,17 +281,17 @@ Calc(select=[b, y]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonAggregateTest.xml index d5f863031770e..d63eebb0dc819 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonAggregateTest.xml @@ -21,7 +21,7 @@ limitations under the License. @@ -30,7 +30,7 @@ PythonGroupAggregate(groupBy=[b], select=[b, PandasAggregateFunction(a, c) AS EX +- Exchange(distribution=[keep_input_as_is[hash[b]]]) +- Sort(orderBy=[b ASC]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -39,14 +39,14 @@ PythonGroupAggregate(groupBy=[b], select=[b, PandasAggregateFunction(a, c) AS EX diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonCalcTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonCalcTest.xml index 3214c45b9446a..b77ddaef7a60e 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonCalcTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonCalcTest.xml @@ -23,14 +23,14 @@ limitations under the License. diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonGroupWindowAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonGroupWindowAggregateTest.xml index 0b4c66833ee93..e77431b099d0f 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonGroupWindowAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonGroupWindowAggregateTest.xml @@ -21,7 +21,7 @@ limitations under the License. @@ -31,7 +31,7 @@ Calc(select=[b, EXPR$0, EXPR$1, EXPR$2]) +- Exchange(distribution=[keep_input_as_is[hash[b]]]) +- Sort(orderBy=[b ASC, rowtime ASC]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, rowtime)]]], fields=[a, b, c, rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, rowtime]) ]]> @@ -40,7 +40,7 @@ Calc(select=[b, EXPR$0, EXPR$1, EXPR$2]) @@ -50,7 +50,7 @@ Calc(select=[b, EXPR$0, EXPR$1, EXPR$2]) +- Exchange(distribution=[keep_input_as_is[hash[b]]]) +- Sort(orderBy=[b ASC, rowtime ASC]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, rowtime)]]], fields=[a, b, c, rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, rowtime]) ]]> @@ -59,7 +59,7 @@ Calc(select=[b, EXPR$0, EXPR$1, EXPR$2]) @@ -68,7 +68,7 @@ Calc(select=[EXPR$0, EXPR$1, EXPR$2]) +- PythonGroupWindowAggregate(window=[SlidingGroupWindow('w, rowtime, 5, 2)], properties=[EXPR$0, EXPR$1], select=[PandasAggregateFunction(a, c) AS EXPR$2]) +- Sort(orderBy=[rowtime ASC]) +- Exchange(distribution=[single]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, rowtime)]]], fields=[a, b, c, rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, rowtime]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonOverWindowAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonOverWindowAggregateTest.xml index 191c5a555fabb..db2295d8a931a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonOverWindowAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonOverWindowAggregateTest.xml @@ -20,7 +20,7 @@ limitations under the License. @@ -30,7 +30,7 @@ Calc(select=[b, w0$o0 AS _c1]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[b ASC, rowtime ASC]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, rowtime)]]], fields=[a, b, c, rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, rowtime]) ]]> @@ -38,7 +38,7 @@ Calc(select=[b, w0$o0 AS _c1]) @@ -48,7 +48,7 @@ Calc(select=[b, w0$o0 AS _c1]) +- Exchange(distribution=[forward]) +- Sort(orderBy=[b ASC, rowtime ASC]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, rowtime)]]], fields=[a, b, c, rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, rowtime]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.xml index 7d800a1411de6..b58d6dc1606c2 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.xml @@ -23,8 +23,8 @@ LogicalProject(a=[$1], b=[$0], c=[$2]) +- LogicalAggregate(group=[{1}], EXPR$0=[SUM($0)], EXPR$1=[COUNT($2)]) +- LogicalFilter(condition=[>($0, 0)]) +- LogicalMinus(all=[true]) - :- LogicalTableScan(table=[[default_catalog, default_database, left, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, right, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, left]]) + +- LogicalTableScan(table=[[default_catalog, default_database, right]]) ]]> @@ -41,9 +41,9 @@ Calc(select=[EXPR$0 AS a, b, EXPR$1 AS c]) +- LocalHashAggregate(groupBy=[a, b, c], select=[a, b, c, Partial_SUM(vcol_marker) AS sum$0]) +- Union(all=[true], union=[a, b, c, vcol_marker]) :- Calc(select=[a, b, c, 1 AS vcol_marker], where=[(a > 0)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, left, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, left]], fields=[a, b, c]) +- Calc(select=[a, b, c, -1 AS vcol_marker], where=[(a > 0)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, right, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, right]], fields=[a, b, c]) ]]> @@ -54,8 +54,8 @@ LogicalProject(a=[$1], b=[$0], c=[$2]) +- LogicalAggregate(group=[{1}], EXPR$0=[SUM($0)], EXPR$1=[COUNT($2)]) +- LogicalFilter(condition=[>($0, 0)]) +- LogicalUnion(all=[true]) - :- LogicalTableScan(table=[[default_catalog, default_database, left, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, right, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, left]]) + +- LogicalTableScan(table=[[default_catalog, default_database, right]]) ]]> @@ -66,9 +66,9 @@ Calc(select=[EXPR$0 AS a, b, EXPR$1 AS c]) +- LocalHashAggregate(groupBy=[b], select=[b, Partial_SUM(a) AS sum$0, Partial_COUNT(c) AS count$1]) +- Union(all=[true], union=[a, b, c]) :- Calc(select=[a, b, c], where=[(a > 0)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, left, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, left]], fields=[a, b, c]) +- Calc(select=[a, b, c], where=[(a > 0)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, right, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, right]], fields=[a, b, c]) ]]> @@ -78,17 +78,17 @@ Calc(select=[EXPR$0 AS a, b, EXPR$1 AS c]) LogicalFilter(condition=[IN($2, { LogicalProject(a1=[AS($0, _UTF-16LE'a1')]) LogicalFilter(condition=[=($1, _UTF-16LE'two')]) - LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a, b, c)]]]) + LogicalTableScan(table=[[default_catalog, default_database, A]]) })]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, A]]) ]]> @@ -116,9 +116,9 @@ Calc(select=[(b = 1972-02-22 07:12:00.333) AS b2]) LogicalProject(b=[$1], c=[$2]) +- LogicalMinus(all=[true]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, left, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, left]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, right, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, right]]) ]]> @@ -131,9 +131,9 @@ Calc(select=[b0 AS b, c0 AS c]) +- LocalHashAggregate(groupBy=[b, c], select=[b, c, Partial_SUM(vcol_marker) AS sum$0]) +- Union(all=[true], union=[b, c, vcol_marker]) :- Calc(select=[b, c, 1 AS vcol_marker]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, left, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, left]], fields=[a, b, c]) +- Calc(select=[b, c, -1 AS vcol_marker]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, right, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, right]], fields=[a, b, c]) ]]> @@ -142,16 +142,16 @@ Calc(select=[b0 AS b, c0 AS c]) ($2, 0), $1, null:RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) CHARACTER SET "UTF-16LE" _2))]) - +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, A]]) ]]> 0), b, null:RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2)) AS _c0]) +- Reused(reference_id=[1]) ]]> @@ -163,18 +163,18 @@ Union(all=[true], union=[a]) LogicalProject(b=[$1], c=[$2]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, left, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, left]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, right, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, right]]) ]]> @@ -183,16 +183,16 @@ Union(all=[true], union=[b, c]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/common/PartialInsertTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/common/PartialInsertTest.xml index 2064ed8df954d..cec9966bb100b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/common/PartialInsertTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/common/PartialInsertTest.xml @@ -22,21 +22,21 @@ limitations under the License. LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[4],[0],[3]], fields=[a, EXPR$1, d, e, EXPR$4, EXPR$5]) +- LogicalProject(a=[$0], EXPR$1=[null:VARCHAR(2147483647) CHARACTER SET "UTF-16LE"], d=[$3], e=[$4], EXPR$4=[null:BIGINT], EXPR$5=[null:INTEGER]) +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) == Optimized Physical Plan == Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[4],[0],[3]], fields=[a, EXPR$1, d, e, EXPR$4, EXPR$5]) +- Calc(select=[a, null:VARCHAR(2147483647) CHARACTER SET "UTF-16LE" AS EXPR$1, d, e, null:BIGINT AS EXPR$4, null:INTEGER AS EXPR$5]) +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) +- Exchange(distribution=[hash[a, b, c, d, e]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) == Optimized Execution Plan == Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[4],[0],[3]], fields=[a, EXPR$1, d, e, EXPR$4, EXPR$5]) +- Calc(select=[a, null:VARCHAR(2147483647) AS EXPR$1, d, e, null:BIGINT AS EXPR$4, null:INTEGER AS EXPR$5]) +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) +- Exchange(distribution=[hash[a, b, c, d, e]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -51,10 +51,10 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetCol +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[123], EXPR$5=[456]) : +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[456], EXPR$5=[789]) +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -66,12 +66,12 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ : +- HashAggregate(isMerge=[true], groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) : +- Exchange(distribution=[hash[a, b, c, d, e]]) : +- LocalHashAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Calc(select=[a, c, d, e, CAST(456 AS BIGINT) AS f, CAST(789 AS INTEGER) AS g]) +- HashAggregate(isMerge=[true], groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) +- Exchange(distribution=[hash[a, b, c, d, e]]) +- LocalHashAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -81,7 +81,7 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[4],[0],[3]], fields=[a, EXPR$1, d, e, EXPR$4, EXPR$5]) +- LogicalProject(a=[$0], EXPR$1=[null:VARCHAR(2147483647) CHARACTER SET "UTF-16LE"], d=[$3], e=[$4], EXPR$4=[null:BIGINT], EXPR$5=[null:INTEGER]) +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) == Optimized Physical Plan == Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[4],[0],[3]], fields=[a, EXPR$1, d, e, EXPR$4, EXPR$5]) @@ -90,7 +90,7 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ +- HashAggregate(isMerge=[true], groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) +- Exchange(distribution=[hash[a, b, c, d, e]]) +- LocalHashAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) == Optimized Execution Plan == Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[4],[0],[3]], fields=[a, EXPR$1, d, e, EXPR$4, EXPR$5]) @@ -99,7 +99,7 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ +- HashAggregate(isMerge=[true], groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) +- Exchange(distribution=[hash[a, b, c, d, e]]) +- LocalHashAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -112,7 +112,7 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ LogicalSink(table=[default_catalog.default_database.sink], targetColumns=[[1],[4],[0],[6],[5],[2],[3]], fields=[a, b, c, d, e, f, g]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[123:BIGINT], g=[456]) +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -121,7 +121,7 @@ Sink(table=[default_catalog.default_database.sink], targetColumns=[[1],[4],[0],[ +- Calc(select=[a, b, c, d, e, 123 AS f, 456 AS g]) +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) +- Exchange(distribution=[hash[a, b, c, d, e]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -134,7 +134,7 @@ Sink(table=[default_catalog.default_database.sink], targetColumns=[[1],[4],[0],[ LogicalSink(table=[default_catalog.default_database.sink], targetColumns=[[1],[4],[0],[6],[5],[2],[3]], fields=[a, b, c, d, e, f, g]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[123:BIGINT], g=[456]) +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -144,7 +144,7 @@ Sink(table=[default_catalog.default_database.sink], targetColumns=[[1],[4],[0],[ +- HashAggregate(isMerge=[true], groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) +- Exchange(distribution=[hash[a, b, c, d, e]]) +- LocalHashAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -157,7 +157,7 @@ Sink(table=[default_catalog.default_database.sink], targetColumns=[[1],[4],[0],[ LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[4],[0],[6],[5],[2],[3]], fields=[a, c, d, e, f, g]) +- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], f=[123:BIGINT], g=[456]) +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -166,7 +166,7 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ +- Calc(select=[a, c, d, e, 123 AS f, 456 AS g]) +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) +- Exchange(distribution=[hash[a, b, c, d, e]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -179,7 +179,7 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[4],[0],[6],[5],[2],[3]], fields=[a, c, d, e, f, g]) +- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], f=[123:BIGINT], g=[456]) +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -190,7 +190,7 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ +- HashAggregate(isMerge=[true], groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) +- Exchange(distribution=[hash[a, b, c, d, e]]) +- LocalHashAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -205,10 +205,10 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetCol +- LogicalMinus(all=[true]) :- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[123], EXPR$5=[456]) : +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[456], EXPR$5=[789]) +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -223,11 +223,11 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ :- Calc(select=[a, c, d, e, CAST(123 AS BIGINT) AS f, CAST(456 AS INTEGER) AS g, 1 AS vcol_marker]) : +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) : +- Exchange(distribution=[hash[a, b, c, d, e]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Calc(select=[a, c, d, e, CAST(123 AS BIGINT) AS f, CAST(456 AS INTEGER) AS g, -1 AS vcol_marker]) +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) +- Exchange(distribution=[hash[a, b, c, d, e]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -242,10 +242,10 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetCol +- LogicalMinus(all=[true]) :- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[123], EXPR$5=[456]) : +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[456], EXPR$5=[789]) +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -263,12 +263,12 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ : +- HashAggregate(isMerge=[true], groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) : +- Exchange(distribution=[hash[a, b, c, d, e]]) : +- LocalHashAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Calc(select=[a, c, d, e, CAST(123 AS BIGINT) AS f, CAST(456 AS INTEGER) AS g, -1 AS vcol_marker]) +- HashAggregate(isMerge=[true], groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) +- Exchange(distribution=[hash[a, b, c, d, e]]) +- LocalHashAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -283,10 +283,10 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetCol +- LogicalIntersect(all=[true]) :- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[123], EXPR$5=[456]) : +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[456], EXPR$5=[789]) +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -301,11 +301,11 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ :- Calc(select=[a, c, d, e, CAST(123 AS BIGINT) AS f, CAST(456 AS INTEGER) AS g, true AS vcol_left_marker, null:BOOLEAN AS vcol_right_marker]) : +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) : +- Exchange(distribution=[hash[a, b, c, d, e]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Calc(select=[a, c, d, e, CAST(456 AS BIGINT) AS f, CAST(789 AS INTEGER) AS g, null:BOOLEAN AS vcol_left_marker, true AS vcol_right_marker]) +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) +- Exchange(distribution=[hash[a, b, c, d, e]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -320,10 +320,10 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetCol +- LogicalIntersect(all=[true]) :- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[123], EXPR$5=[456]) : +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[456], EXPR$5=[789]) +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -341,12 +341,12 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ : +- HashAggregate(isMerge=[true], groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) : +- Exchange(distribution=[hash[a, b, c, d, e]]) : +- LocalHashAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Calc(select=[a, c, d, e, CAST(456 AS BIGINT) AS f, CAST(789 AS INTEGER) AS g, null:BOOLEAN AS vcol_left_marker, true AS vcol_right_marker]) +- HashAggregate(isMerge=[true], groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) +- Exchange(distribution=[hash[a, b, c, d, e]]) +- LocalHashAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -360,7 +360,7 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetCol +- LogicalProject(a=[$0], c=[$1], d=[$2], e=[$3], f=[CAST($4):BIGINT], g=[CAST($5):INTEGER]) +- LogicalSort(sort0=[$0], sort1=[$3], sort2=[$1], sort3=[$2], dir0=[ASC-nulls-first], dir1=[ASC-nulls-first], dir2=[ASC-nulls-first], dir3=[ASC-nulls-first]) +- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[123], EXPR$5=[456]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -369,7 +369,7 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ +- Calc(select=[a, c, d, e, CAST(123 AS BIGINT) AS f, CAST(456 AS INTEGER) AS g]) +- Sort(orderBy=[a ASC, e ASC, c ASC, d ASC]) +- Exchange(distribution=[single]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -383,7 +383,7 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetCol +- LogicalProject(a=[$0], c=[$1], d=[$2], e=[$3], f=[CAST($4):BIGINT], g=[CAST($5):INTEGER]) +- LogicalSort(sort0=[$0], sort1=[$3], sort2=[$1], sort3=[$2], dir0=[ASC-nulls-first], dir1=[ASC-nulls-first], dir2=[ASC-nulls-first], dir3=[ASC-nulls-first]) +- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[123], EXPR$5=[456]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -392,7 +392,7 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ +- Calc(select=[a, c, d, e, CAST(123 AS BIGINT) AS f, CAST(456 AS INTEGER) AS g]) +- Sort(orderBy=[a ASC, e ASC, c ASC, d ASC]) +- Exchange(distribution=[single]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -404,14 +404,14 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ @@ -423,14 +423,14 @@ Sink(table=[default_catalog.default_database.metadata_sink], targetColumns=[[0], @@ -445,10 +445,10 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetCol +- LogicalUnion(all=[false]) :- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[123], EXPR$5=[456]) : +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[456], EXPR$5=[789]) +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -461,11 +461,11 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ :- Calc(select=[a, c, d, e, 123 AS EXPR$4, 456 AS EXPR$5]) : +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) : +- Exchange(distribution=[hash[a, b, c, d, e]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Calc(select=[a, c, d, e, 456 AS EXPR$4, 789 AS EXPR$5]) +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) +- Exchange(distribution=[hash[a, b, c, d, e]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -480,10 +480,10 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetCol +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[123], EXPR$5=[456]) : +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[456], EXPR$5=[789]) +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -493,11 +493,11 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ :- Calc(select=[a, c, d, e, CAST(123 AS BIGINT) AS f, CAST(456 AS INTEGER) AS g]) : +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) : +- Exchange(distribution=[hash[a, b, c, d, e]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Calc(select=[a, c, d, e, CAST(456 AS BIGINT) AS f, CAST(789 AS INTEGER) AS g]) +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) +- Exchange(distribution=[hash[a, b, c, d, e]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -513,13 +513,13 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetCol :- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[123], EXPR$5=[456]) : : +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[456], EXPR$5=[789]) : +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[456], EXPR$5=[123]) +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -532,17 +532,17 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ : : +- HashAggregate(isMerge=[true], groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) : : +- Exchange(distribution=[hash[a, b, c, d, e]]) : : +- LocalHashAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + : : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) : +- Calc(select=[a, c, d, e, CAST(456 AS BIGINT) AS f, CAST(789 AS INTEGER) AS g]) : +- HashAggregate(isMerge=[true], groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) : +- Exchange(distribution=[hash[a, b, c, d, e]]) : +- LocalHashAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Calc(select=[a, c, d, e, CAST(456 AS BIGINT) AS f, CAST(123 AS INTEGER) AS g]) +- HashAggregate(isMerge=[true], groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) +- Exchange(distribution=[hash[a, b, c, d, e]]) +- LocalHashAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -557,10 +557,10 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetCol +- LogicalUnion(all=[false]) :- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[123], EXPR$5=[456]) : +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[456], EXPR$5=[789]) +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -576,12 +576,12 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ : +- HashAggregate(isMerge=[true], groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) : +- Exchange(distribution=[hash[a, b, c, d, e]]) : +- LocalHashAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Calc(select=[a, c, d, e, 456 AS EXPR$4, 789 AS EXPR$5]) +- HashAggregate(isMerge=[true], groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) +- Exchange(distribution=[hash[a, b, c, d, e]]) +- LocalHashAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -597,13 +597,13 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetCol :- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[123], EXPR$5=[456]) : : +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[456], EXPR$5=[789]) : +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], c=[$2], d=[$3], e=[$4], EXPR$4=[456], EXPR$5=[123]) +- LogicalAggregate(group=[{0, 1, 2, 3, 4}]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -614,15 +614,15 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[ : :- Calc(select=[a, c, d, e, CAST(123 AS BIGINT) AS f, CAST(456 AS INTEGER) AS g]) : : +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) : : +- Exchange(distribution=[hash[a, b, c, d, e]]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + : : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) : +- Calc(select=[a, c, d, e, CAST(456 AS BIGINT) AS f, CAST(789 AS INTEGER) AS g]) : +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) : +- Exchange(distribution=[hash[a, b, c, d, e]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Calc(select=[a, c, d, e, CAST(456 AS BIGINT) AS f, CAST(123 AS INTEGER) AS g]) +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e]) +- Exchange(distribution=[hash[a, b, c, d, e]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPruneAggregateCallRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPruneAggregateCallRuleTest.xml index 619150eaf1e1d..6ee7786d08166 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPruneAggregateCallRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPruneAggregateCallRuleTest.xml @@ -25,7 +25,7 @@ limitations under the License. LogicalProject(EXPR$0=[1]) +- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)], EXPR$1=[COUNT()]) +- LogicalProject(a1=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) ]]> @@ -33,7 +33,7 @@ LogicalProject(EXPR$0=[1]) LogicalCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1]) +- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)]) +- LogicalCalc(expr#0..3=[{inputs}], a1=[$t0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) ]]> @@ -47,14 +47,14 @@ LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3]) +- LogicalFilter(condition=[EXISTS({ LogicalAggregate(group=[{}], EXPR$0=[SUM($0)], EXPR$1=[COUNT()]) LogicalProject(a1=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T1]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2, d2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -69,14 +69,14 @@ LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3]) LogicalAggregate(group=[{}], EXPR$0=[SUM($0)], EXPR$1=[COUNT()]) LogicalProject(a1=[$0]) LogicalFilter(condition=[=(1, 2)]) - LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T1]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2, d2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -89,7 +89,7 @@ LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTab LogicalProject(EXPR$0=[1]) +- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)]) +- LogicalProject(a1=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) ]]> @@ -97,7 +97,7 @@ LogicalProject(EXPR$0=[1]) LogicalCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1]) +- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)]) +- LogicalCalc(expr#0..3=[{inputs}], a1=[$t0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) ]]> @@ -111,14 +111,14 @@ LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3]) +- LogicalFilter(condition=[EXISTS({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalProject($f0=[0]) - LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T1]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2, d2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -133,14 +133,14 @@ LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3]) LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalProject($f0=[0]) LogicalFilter(condition=[=(1, 2)]) - LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T1]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2, d2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -157,14 +157,14 @@ WHERE c1 > 10 LogicalProject(a1=[$0], c1=[$2]) +- LogicalFilter(condition=[>($2, 10)]) +- LogicalAggregate(group=[{0, 1}], c1=[COUNT($2)], d1=[SUM($3)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) ]]> ($t1, $t2)], proj#0..1=[{exprs}], $condition=[$t3]) +- LogicalAggregate(group=[{0}], c1=[COUNT($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) ]]> @@ -181,14 +181,14 @@ WHERE c1 > 10 LogicalProject(c1=[$2], a1=[$0]) +- LogicalFilter(condition=[>($2, 10)]) +- LogicalAggregate(group=[{0, 1}], c1=[COUNT($2)], d1=[SUM($3)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) ]]> ($t1, $t2)], c1=[$t1], a1=[$t0], $condition=[$t3]) +- LogicalAggregate(group=[{0}], c1=[COUNT($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) ]]> @@ -203,13 +203,13 @@ SELECT a1, c1 FROM @@ -224,14 +224,14 @@ SELECT c1, a1 FROM @@ -248,14 +248,14 @@ WHERE d2 > 0 LogicalProject(a2=[$0], b2=[$1], d2=[$3]) +- LogicalFilter(condition=[>($3, 0)]) +- LogicalAggregate(group=[{0, 1}], c2=[COUNT($2)], d2=[SUM($3)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2, d2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> ($t2, $t3)], proj#0..2=[{exprs}], $condition=[$t4]) +- LogicalAggregate(group=[{0, 1}], d2=[SUM($3)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2, d2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -272,14 +272,14 @@ WHERE d2 > 0 LogicalProject(b2=[$1], a2=[$0], d2=[$3]) +- LogicalFilter(condition=[>($3, 0)]) +- LogicalAggregate(group=[{0, 1}], c2=[COUNT($2)], d2=[SUM($3)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2, d2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> ($t2, $t3)], b2=[$t1], a2=[$t0], d2=[$t2], $condition=[$t4]) +- LogicalAggregate(group=[{0, 1}], d2=[SUM($3)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2, d2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -294,13 +294,13 @@ SELECT a2, b2, d2 FROM @@ -315,14 +315,14 @@ SELECT b2, a2, d2 FROM @@ -337,14 +337,14 @@ SELECT a2 as a, b2, d2 FROM diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRuleTest.xml index 7dd5276127300..6aaa9a6e855d1 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRuleTest.xml @@ -25,7 +25,7 @@ limitations under the License. LogicalProject(a=[$0], b=[$1], c=[$2], x=[$3], y=[$4]) +- LogicalFilter(condition=[AND(=($3, $0), =(pyFunc($3, $3), 2), =(+($4, 1), *($4, $4)))]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{0, 1}]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableFunctionScan(invocation=[func(*($cor0.a, $cor0.a), $cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER x, INTEGER y)]) ]]> @@ -35,7 +35,7 @@ FlinkLogicalCalc(select=[a, b, c, x, y], where=[AND(=(f0, 2), =(+(y, 1), *(y, y) +- FlinkLogicalCalc(select=[a, b, c, x, y, pyFunc(x, x) AS f0]) +- FlinkLogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{0, 1}]) :- FlinkLogicalCalc(select=[a, b, c, *(a, a) AS f0]) - : +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) +- FlinkLogicalTableFunctionScan(invocation=[func($3, $1)], rowType=[RecordType:peek_no_expand(INTEGER x, INTEGER y)]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcRankTransposeRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcRankTransposeRuleTest.xml index 282ffd96762bf..7593a0a702250 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcRankTransposeRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcRankTransposeRuleTest.xml @@ -36,7 +36,7 @@ LogicalProject(category=[$0], max_price=[$2], rank_num=[$3]) +- LogicalFilter(condition=[<=($3, 3)]) +- LogicalProject(category=[$0], shopId=[$1], max_price=[$2], rank_num=[ROW_NUMBER() OVER (PARTITION BY $0 ORDER BY $2 NULLS FIRST)]) +- LogicalAggregate(group=[{0, 1}], max_price=[MAX($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(category, shopId, price)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -44,7 +44,7 @@ LogicalProject(category=[$0], max_price=[$2], rank_num=[$3]) FlinkLogicalCalc(select=[category, max_price, w0$o0]) +- FlinkLogicalRank(rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=3], partitionBy=[category], orderBy=[max_price ASC], select=[category, shopId, max_price, w0$o0]) +- FlinkLogicalAggregate(group=[{0, 1}], max_price=[MAX($2)]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(category, shopId, price)]]], fields=[category, shopId, price]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, T]], fields=[category, shopId, price]) ]]> @@ -202,7 +202,7 @@ LogicalProject(category=[$0], max_price=[$2], rank_num=[$4]) +- LogicalFilter(condition=[<=($4, 3)]) +- LogicalProject(category=[$0], shopId=[$1], max_price=[$2], min_price=[$3], rank_num=[ROW_NUMBER() OVER (PARTITION BY $0 ORDER BY $2 NULLS FIRST)]) +- LogicalAggregate(group=[{0, 1}], max_price=[MAX($2)], min_price=[MIN($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(category, shopId, price)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -211,7 +211,7 @@ FlinkLogicalCalc(select=[category, max_price, w0$o0]) +- FlinkLogicalRank(rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=3], partitionBy=[category], orderBy=[max_price ASC], select=[category, shopId, max_price, w0$o0]) +- FlinkLogicalCalc(select=[category, shopId, max_price]) +- FlinkLogicalAggregate(group=[{0, 1}], max_price=[MAX($2)], min_price=[MIN($2)]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(category, shopId, price)]]], fields=[category, shopId, price]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, T]], fields=[category, shopId, price]) ]]> @@ -236,7 +236,7 @@ LogicalProject(category=[$0], shopId=[$1], max_price=[$2], rank_num=[$3]) +- LogicalFilter(condition=[<=($3, 3)]) +- LogicalProject(category=[$0], shopId=[$1], max_price=[$2], rank_num=[ROW_NUMBER() OVER (PARTITION BY $0 ORDER BY $2 NULLS FIRST)]) +- LogicalAggregate(group=[{0, 1}], max_price=[MAX($2)], min_price=[MIN($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(category, shopId, price)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -244,7 +244,7 @@ LogicalProject(category=[$0], shopId=[$1], max_price=[$2], rank_num=[$3]) FlinkLogicalRank(rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=3], partitionBy=[category], orderBy=[max_price ASC], select=[category, shopId, max_price, w0$o0]) +- FlinkLogicalCalc(select=[category, shopId, max_price]) +- FlinkLogicalAggregate(group=[{0, 1}], max_price=[MAX($2)], min_price=[MIN($2)]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(category, shopId, price)]]], fields=[category, shopId, price]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, T]], fields=[category, shopId, price]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRuleTest.xml index 739020c84b434..ead65f5a84ea0 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRuleTest.xml @@ -24,14 +24,14 @@ limitations under the License. @@ -43,14 +43,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -62,14 +62,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -81,14 +81,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -100,14 +100,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -119,14 +119,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -138,14 +138,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -157,14 +157,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -176,14 +176,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -206,14 +206,14 @@ SELECT * from MyTable where e in ( @@ -225,14 +225,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) ($0, 1), <>($0, 2), <>($0, 3), <>($0, 4)))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -244,14 +244,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) ($0, 1), <>($0, 2), <>($0, 3), <>($0, 4)))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -263,14 +263,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) ($0, 1), <>($0, 2), <>($0, 3), <>($0, 4), =($2, 1)), =($1, 3), =($1, 4), =($2, 1))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -282,14 +282,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) ($0, 1), <>($0, 2), <>($0, 3), <>($0, 4))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -301,14 +301,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) ($3, 1), <>($3, 2), <>($3, 3), <>($3, 4), <>($3, 5), <>($3, 6), <>($3, 7), <>($3, 8), <>($3, 9), <>($3, 10), <>($3, 11), <>($3, 12), <>($3, 13), <>($3, 14), <>($3, 15), <>($3, 16), <>($3, 17), <>($3, 18), <>($3, 19), <>($3, 20))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -320,14 +320,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) ($0, 1), <>($0, 2), <>($0, 3), <>($0, 4), =($0, 5))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -339,14 +339,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) ($3, 1), <>($3, 2), <>($3, 3), <>($3, 4), <>($3, 5), <>($3, 6), <>($3, 7), <>($3, 8), <>($3, 9), <>($3, 10), <>($3, 11), <>($3, 12), <>($3, 13), <>($3, 14), <>($3, 15), <>($3, 16), <>($3, 17), <>($3, 18), <>($3, 19))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> ($3, 1), <>($3, 2), <>($3, 3), <>($3, 4), <>($3, 5), <>($3, 6), <>($3, 7), <>($3, 8), <>($3, 9), <>($3, 10), <>($3, 11), <>($3, 12), <>($3, 13), <>($3, 14), <>($3, 15), <>($3, 16), <>($3, 17), <>($3, 18), <>($3, 19))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -358,14 +358,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) ($0, 1), <>($0, 2), <>($0, 3))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> ($0, 1), <>($0, 2), <>($0, 3))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -377,14 +377,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) ($0, 1), <>($0, 2), <>($0, 3), <>($0, 4), =($1, 1))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -396,14 +396,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) ($0, 1), <>($0, 2), =($1, 1), <>($0, 3), <>($0, 4))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -415,14 +415,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) ($0, 1), <>($0, 2), <>($0, 3), <>($0, 4)), =($1, 1))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -434,14 +434,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) ($0, 1), <>($0, 2), <>($0, 3), <>($0, 4), =($1, 1))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> ($0, 1), <>($0, 2), <>($0, 3), <>($0, 4), =($1, 1))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -453,14 +453,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -472,14 +472,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) ($0, 1), <>($0, 2), <>($0, 3), <>($0, 4)), =($1, 1))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -491,14 +491,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/DecomposeGroupingSetsRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/DecomposeGroupingSetsRuleTest.xml index 84e22439ed46d..f12b0b46225a6 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/DecomposeGroupingSetsRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/DecomposeGroupingSetsRuleTest.xml @@ -28,11 +28,11 @@ LogicalUnion(all=[true]) :- LogicalProject(a=[$0], g=[0:BIGINT], c=[$1]) : +- LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()]) : +- LogicalProject(a=[$0]) -: +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], g=[1:BIGINT], c=[$1]) +- LogicalAggregate(group=[{0}], groups=[[{}]], c=[COUNT()]) +- LogicalProject(a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -42,11 +42,11 @@ FlinkLogicalUnion(all=[true]) : +- FlinkLogicalAggregate(group=[{0, 1}], c=[COUNT()]) : +- FlinkLogicalExpand(projects=[{a, 0 AS $e}, {null AS a, 1 AS $e}]) : +- FlinkLogicalCalc(select=[a]) -: +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) +- FlinkLogicalCalc(select=[a, 1 AS g, c]) +- FlinkLogicalAggregate(group=[{0}], groups=[[{}]], c=[COUNT()]) +- FlinkLogicalCalc(select=[a]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -70,7 +70,7 @@ FROM MyTable LogicalProject(b=[$0], c=[$1], a=[$2], g=[0:BIGINT], gb=[$3], gc=[$4], gib=[$5], gic=[$6], gid=[$7]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], a=[AVG($2)], gb=[GROUPING($0)], gc=[GROUPING($1)], gib=[GROUPING_ID($0)], gic=[GROUPING_ID($1)], gid=[GROUPING_ID($0, $1)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -78,7 +78,7 @@ LogicalProject(b=[$0], c=[$1], a=[$2], g=[0:BIGINT], gb=[$3], gc=[$4], gib=[$5], FlinkLogicalCalc(select=[b, c, a, 0 AS g, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS gb, CASE(=($e, 0), 0, =($e, 1), 1, =($e, 2), 0, 1) AS gc, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS gib, CASE(=($e, 0), 0, =($e, 1), 1, =($e, 2), 0, 1) AS gic, CASE(=($e, 0), 0, =($e, 1), 1, =($e, 2), 2, 3) AS gid]) +- FlinkLogicalAggregate(group=[{1, 2, 3}], a=[AVG($0)]) +- FlinkLogicalExpand(projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -99,7 +99,7 @@ FROM MyTable LogicalProject(b=[$0], a=[$1], g=[0:BIGINT], gb=[$2], gib=[$3]) +- LogicalAggregate(group=[{0}], a=[AVG($1)], gb=[GROUPING($0)], gib=[GROUPING_ID($0)]) +- LogicalProject(b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -107,7 +107,7 @@ LogicalProject(b=[$0], a=[$1], g=[0:BIGINT], gb=[$2], gib=[$3]) FlinkLogicalCalc(select=[b, a, 0 AS g, 0 AS gb, 0 AS gib]) +- FlinkLogicalAggregate(group=[{0}], a=[AVG($1)]) +- FlinkLogicalCalc(select=[b, a]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -123,7 +123,7 @@ GROUP BY GROUPING SETS (b, c) LogicalProject(b=[$0], c=[$1], a=[$2], g=[0:BIGINT]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}]], a=[AVG($2)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -131,7 +131,7 @@ LogicalProject(b=[$0], c=[$1], a=[$2], g=[0:BIGINT]) FlinkLogicalCalc(select=[b, c, a, 0 AS g]) +- FlinkLogicalAggregate(group=[{1, 2, 3}], a=[AVG($0)]) +- FlinkLogicalExpand(projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -146,7 +146,7 @@ GROUP BY GROUPING SETS ((a, b), (a, c)) @@ -154,7 +154,7 @@ LogicalProject(a=[$3], b=[$4], c=[$5]) FlinkLogicalCalc(select=[a_0 AS a, b_0 AS b, c_0 AS c]) +- FlinkLogicalAggregate(group=[{0, 1, 2, 3}], a=[COUNT($0)], b=[COUNT($4)], c=[COUNT($5)]) +- FlinkLogicalExpand(projects=[{a, b, null AS c, 1 AS $e, b AS b_0, c AS c_0}, {a, null AS b, c, 2 AS $e, b AS b_0, c AS c_0}]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -178,7 +178,7 @@ FROM MyTable LogicalProject(b=[$0], c=[$1], a=[$2], g=[0:BIGINT], gb=[$3], gc=[$4], gib=[$5], gic=[$6], gid=[$7]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], a=[AVG($2)], gb=[GROUPING($0)], gc=[GROUPING($1)], gib=[GROUPING_ID($0)], gic=[GROUPING_ID($1)], gid=[GROUPING_ID($0, $1)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -186,7 +186,7 @@ LogicalProject(b=[$0], c=[$1], a=[$2], g=[0:BIGINT], gb=[$3], gc=[$4], gib=[$5], FlinkLogicalCalc(select=[b, c, a, 0 AS g, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS gb, CASE(=($e, 0), 0, 1) AS gc, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS gib, CASE(=($e, 0), 0, 1) AS gic, CASE(=($e, 0), 0, =($e, 1), 1, 3) AS gid]) +- FlinkLogicalAggregate(group=[{1, 2, 3}], a=[AVG($0)]) +- FlinkLogicalExpand(projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -201,7 +201,7 @@ SELECT count(b) as b, count(c) as c FROM MyTable GROUP BY GROUPING SETS (b, c) LogicalProject(b=[$2], c=[$3]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}]], b=[COUNT($0)], c=[COUNT($1)]) +- LogicalProject(b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -210,7 +210,7 @@ FlinkLogicalCalc(select=[b_0 AS b, c_0 AS c]) +- FlinkLogicalAggregate(group=[{0, 1, 2}], b=[COUNT($3)], c=[COUNT($4)]) +- FlinkLogicalExpand(projects=[{b, null AS c, 1 AS $e, b AS b_0, c AS c_0}, {null AS b, c, 2 AS $e, b AS b_0, c AS c_0}]) +- FlinkLogicalCalc(select=[b, c]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.xml index 43d644f6fae11..b07d045d3704e 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.xml @@ -23,14 +23,14 @@ limitations under the License. @@ -41,13 +41,13 @@ Calc(select=[f0 AS EXPR$0, CAST(2 AS INTEGER) AS EXPR$1]) @@ -58,13 +58,13 @@ Calc(select=[RAND() AS EXPR$0, RAND() AS EXPR$1, RAND(1) AS EXPR$2, RAND(1) AS E @@ -75,13 +75,13 @@ Calc(select=[CAST(2 AS INTEGER) AS EXPR$0]) @@ -92,13 +92,13 @@ Calc(select=[CAST(11 AS INTEGER) AS EXPR$0]) @@ -109,13 +109,13 @@ Calc(select=[myUdf(+(1, 1)) AS EXPR$0]) @@ -126,13 +126,13 @@ Calc(select=[MyUdf(1) AS EXPR$0]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRuleTest.xml index 207a20da4edca..7d4852a989f0c 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRuleTest.xml @@ -24,7 +24,7 @@ limitations under the License. @@ -35,7 +35,7 @@ FlinkLogicalCalc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2, EXPR$3]) +- FlinkLogicalAggregate(group=[{0, 1, 2}], EXPR$1=[COUNT($0)], EXPR$2=[SUM($3)]) +- FlinkLogicalExpand(projects=[{a, b, 0 AS $e, b AS b_0}, {a, null AS b, 1 AS $e, b AS b_0}]) +- FlinkLogicalCalc(select=[a, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -48,7 +48,7 @@ COUNT(DISTINCT d) FILTER (WHERE b > 1) FROM MyTable2 GROUP BY a]]> ($1, 1))], d=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -59,7 +59,7 @@ FlinkLogicalCalc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2]) +- FlinkLogicalAggregate(group=[{0, 2, 3, 4}], EXPR$1=[COUNT($1) FILTER $5]) +- FlinkLogicalExpand(projects=[{a, c, $f2, d, 0 AS $e, $f2 AS $f2_0}, {a, c, null AS $f2, null AS d, 3 AS $e, $f2 AS $f2_0}]) +- FlinkLogicalCalc(select=[a, c, IS TRUE(>(b, 1)) AS $f2, d]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -72,7 +72,7 @@ FlinkLogicalCalc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2]) LogicalProject(EXPR$0=[$1], EXPR$1=[$2]) +- LogicalAggregate(group=[{0}], EXPR$0=[SUM(DISTINCT $1)], EXPR$1=[COUNT(DISTINCT $2)]) +- LogicalProject(b=[$1], a=[$0], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -82,7 +82,7 @@ FlinkLogicalCalc(select=[EXPR$0, EXPR$1]) +- FlinkLogicalCalc(select=[a, b, c, =(CASE(=($e, 1), 1, 4), 1) AS $g_1, =(CASE(=($e, 1), 1, 4), 4) AS $g_4]) +- FlinkLogicalAggregate(group=[{0, 1, 2, 3}]) +- FlinkLogicalExpand(projects=[{a, b, null AS c, 1 AS $e}, {null AS a, b, c, 4 AS $e}]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -94,7 +94,7 @@ FlinkLogicalCalc(select=[EXPR$0, EXPR$1]) @@ -104,7 +104,7 @@ FlinkLogicalAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $4], EXPR$1=[SUM($1) +- FlinkLogicalAggregate(group=[{0, 1, 2}], EXPR$2=[MAX($3)], EXPR$3=[MIN($3)]) +- FlinkLogicalExpand(projects=[{a, null AS b, 1 AS $e, a AS a_0}, {null AS a, b, 2 AS $e, a AS a_0}, {null AS a, null AS b, 3 AS $e, a AS a_0}]) +- FlinkLogicalCalc(select=[a, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -117,7 +117,7 @@ FlinkLogicalAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $4], EXPR$1=[SUM($1) LogicalProject(EXPR$0=[$1], EXPR$1=[$2], EXPR$2=[$3], EXPR$3=[$4]) +- LogicalAggregate(group=[{0}], EXPR$0=[COUNT(DISTINCT $1)], EXPR$1=[SUM(DISTINCT $2)], EXPR$2=[MAX($1)], EXPR$3=[MIN($1)]) +- LogicalProject(c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -127,7 +127,7 @@ FlinkLogicalCalc(select=[EXPR$0, EXPR$1, EXPR$2, EXPR$3]) +- FlinkLogicalCalc(select=[a, b, c, EXPR$2, EXPR$3, =(CASE(=($e, 2), 2, =($e, 4), 4, 6), 2) AS $g_2, =(CASE(=($e, 2), 2, =($e, 4), 4, 6), 4) AS $g_4, =(CASE(=($e, 2), 2, =($e, 4), 4, 6), 6) AS $g_6]) +- FlinkLogicalAggregate(group=[{0, 1, 2, 3}], EXPR$2=[MAX($4)], EXPR$3=[MIN($4)]) +- FlinkLogicalExpand(projects=[{a, null AS b, c, 2 AS $e, a AS a_0}, {null AS a, b, c, 4 AS $e, a AS a_0}, {null AS a, null AS b, c, 6 AS $e, a AS a_0}]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -140,7 +140,7 @@ COUNT(DISTINCT c) FILTER (WHERE a > 5) FROM MyTable]]> ($0, 5))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -150,7 +150,7 @@ FlinkLogicalAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $3], EXPR$1=[SUM($1) +- FlinkLogicalAggregate(group=[{0, 1, 2, 3, 4}]) +- FlinkLogicalExpand(projects=[{a, null AS b, null AS c, null AS $f3, 7 AS $e}, {null AS a, b, null AS c, null AS $f3, 11 AS $e}, {null AS a, null AS b, c, $f3, 12 AS $e}]) +- FlinkLogicalCalc(select=[a, b, c, IS TRUE(>(a, 5)) AS $f3]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -163,7 +163,7 @@ FlinkLogicalAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $3], EXPR$1=[SUM($1) LogicalProject(EXPR$0=[$2], EXPR$1=[$3]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}]], EXPR$0=[COUNT(DISTINCT $2)], EXPR$1=[SUM(DISTINCT $3)]) +- LogicalProject(c=[$2], d=[$3], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -175,7 +175,7 @@ FlinkLogicalCalc(select=[EXPR$0, EXPR$1]) +- FlinkLogicalExpand(projects=[{c, d, a, null AS b, $e, 2 AS $e_0}, {c, d, null AS a, b, $e, 4 AS $e_0}]) +- FlinkLogicalExpand(projects=[{c, null AS d, a, b, 1 AS $e}, {null AS c, d, a, b, 2 AS $e}]) +- FlinkLogicalCalc(select=[c, d, a, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -188,7 +188,7 @@ MAX(DISTINCT a) FROM MyTable]]> ($1, 0))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -198,7 +198,7 @@ FlinkLogicalAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $3], EXPR$1=[SUM($0) +- FlinkLogicalAggregate(group=[{0, 1, 2}], EXPR$2=[MAX($3)]) +- FlinkLogicalExpand(projects=[{a, $f1, 0 AS $e, a AS a_0}, {a, null AS $f1, 1 AS $e, a AS a_0}, {null AS a, null AS $f1, 3 AS $e, a AS a_0}]) +- FlinkLogicalCalc(select=[a, IS TRUE(>(b, 0)) AS $f1]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -211,7 +211,7 @@ FlinkLogicalAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $3], EXPR$1=[SUM($0) LogicalProject(EXPR$0=[$2], EXPR$1=[$3], EXPR$2=[$4]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}]], EXPR$0=[COUNT(DISTINCT $2)], EXPR$1=[SUM(DISTINCT $2)], EXPR$2=[MAX($2)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -221,7 +221,7 @@ FlinkLogicalCalc(select=[EXPR$0, EXPR$1, EXPR$2]) +- FlinkLogicalAggregate(group=[{0, 1, 2, 3}]) +- FlinkLogicalExpand(projects=[{b, null AS c, a, 1 AS $e}, {null AS b, c, a, 2 AS $e}]) +- FlinkLogicalCalc(select=[b, c, a]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -234,7 +234,7 @@ SUM(DISTINCT b), COUNT(c) FROM MyTable]]> ($2, 0))], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -245,7 +245,7 @@ FlinkLogicalCalc(select=[EXPR$0, EXPR$1, CASE(IS NOT NULL(EXPR$2), EXPR$2, 0) AS +- FlinkLogicalAggregate(group=[{0, 1, 2, 4}], EXPR$2=[COUNT($3)]) +- FlinkLogicalExpand(projects=[{a, $f1, null AS b, c, 1 AS $e}, {null AS a, null AS $f1, b, c, 6 AS $e}, {null AS a, null AS $f1, null AS b, c, 7 AS $e}]) +- FlinkLogicalCalc(select=[a, IS TRUE(>(c, 0)) AS $f1, b, c]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -258,7 +258,7 @@ FlinkLogicalCalc(select=[EXPR$0, EXPR$1, CASE(IS NOT NULL(EXPR$2), EXPR$2, 0) AS LogicalProject(EXPR$0=[$2], EXPR$1=[$3], EXPR$2=[$4]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}]], EXPR$0=[COUNT(DISTINCT $2)], EXPR$1=[SUM(DISTINCT $3)], EXPR$2=[COUNT($4)]) +- LogicalProject(d=[$3], e=[$4], a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -269,7 +269,7 @@ FlinkLogicalCalc(select=[EXPR$0, EXPR$1, CAST(EXPR$2 AS BIGINT) AS EXPR$2]) +- FlinkLogicalAggregate(group=[{0, 1, 3, 4, 5, 6}], EXPR$2=[COUNT($2)]) +- FlinkLogicalExpand(projects=[{a, null AS b, c, d, e, $e, 8 AS $e_0}, {null AS a, b, c, d, e, $e, 16 AS $e_0}, {null AS a, null AS b, c, d, e, $e, 24 AS $e_0}]) +- FlinkLogicalExpand(projects=[{a, b, c, d, null AS e, 1 AS $e}, {a, b, c, null AS d, e, 2 AS $e}]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -283,7 +283,7 @@ FROM MyTable2 GROUP BY d]]> ($0, 5))], b=[$1], $f6=[IS TRUE(>($1, 3))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -293,7 +293,7 @@ FlinkLogicalAggregate(group=[{0}], EXPR$1=[MIN($3) FILTER $8], EXPR$2=[MIN($4) F +- FlinkLogicalAggregate(group=[{0, 3, 4, 5, 6, 7}], EXPR$1=[MAX($1)], EXPR$2=[MAX($1) FILTER $2]) +- FlinkLogicalExpand(projects=[{d, e, $f2, c, $f4, null AS b, null AS $f6, 3 AS $e}, {d, e, $f2, c, null AS $f4, null AS b, null AS $f6, 7 AS $e}, {d, e, $f2, null AS c, null AS $f4, b, $f6, 12 AS $e}, {d, e, $f2, null AS c, null AS $f4, null AS b, null AS $f6, 15 AS $e}]) +- FlinkLogicalCalc(select=[d, e, IS TRUE(<(a, 10)) AS $f2, c, IS TRUE(>(a, 5)) AS $f4, b, IS TRUE(>(b, 3)) AS $f6]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -306,7 +306,7 @@ COUNT(DISTINCT b) FILTER (WHERE b > 1) FROM MyTable2 GROUP BY d]]> ($0, 0))], b=[$1], $f4=[IS TRUE(>($1, 1))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -316,7 +316,7 @@ FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $4], EXPR$2=[COUNT($ +- FlinkLogicalAggregate(group=[{0, 1, 2, 3, 4, 5}]) +- FlinkLogicalExpand(projects=[{d, c, $f2, null AS b, null AS $f4, 3 AS $e}, {d, c, null AS $f2, null AS b, null AS $f4, 7 AS $e}, {d, null AS c, null AS $f2, b, $f4, 12 AS $e}]) +- FlinkLogicalCalc(select=[d, c, IS TRUE(>(a, 0)) AS $f2, b, IS TRUE(>(b, 1)) AS $f4]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -329,7 +329,7 @@ COUNT(DISTINCT c) FILTER (WHERE a < 10) FROM MyTable2 GROUP BY d]]> ($0, 10))], $f3=[IS TRUE(<($0, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -339,7 +339,7 @@ FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $4], EXPR$2=[COUNT($ +- FlinkLogicalAggregate(group=[{0, 1, 2, 3, 4}]) +- FlinkLogicalExpand(projects=[{d, c, $f2, null AS $f3, 1 AS $e}, {d, c, null AS $f2, $f3, 2 AS $e}, {d, c, null AS $f2, null AS $f3, 3 AS $e}]) +- FlinkLogicalCalc(select=[d, c, IS TRUE(>(a, 10)) AS $f2, IS TRUE(<(a, 10)) AS $f3]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -352,7 +352,7 @@ MAX(e), MIN(e) FROM MyTable2 GROUP BY d]]> ($0, 0))], e=[$4]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -362,7 +362,7 @@ FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $5], EXPR$2=[COUNT($ +- FlinkLogicalAggregate(group=[{0, 1, 2, 4}], EXPR$3=[MAX($3)], EXPR$4=[MIN($3)]) +- FlinkLogicalExpand(projects=[{d, c, $f2, e, 0 AS $e}, {d, c, null AS $f2, e, 1 AS $e}, {d, null AS c, null AS $f2, e, 3 AS $e}]) +- FlinkLogicalCalc(select=[d, c, IS TRUE(>(a, 0)) AS $f2, e]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -374,7 +374,7 @@ FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $5], EXPR$2=[COUNT($ @@ -382,7 +382,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT(DISTINCT $0)]) FlinkLogicalAggregate(group=[{}], EXPR$0=[COUNT($0)]) +- FlinkLogicalAggregate(group=[{0}]) +- FlinkLogicalCalc(select=[a]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -394,7 +394,7 @@ FlinkLogicalAggregate(group=[{}], EXPR$0=[COUNT($0)]) ($0, 0))], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -404,7 +404,7 @@ FlinkLogicalAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $2], EXPR$1=[MIN($1) +- FlinkLogicalAggregate(group=[{0, 1, 3}], EXPR$1=[SUM($2)]) +- FlinkLogicalExpand(projects=[{a, $f1, b, 0 AS $e}, {null AS a, null AS $f1, b, 3 AS $e}]) +- FlinkLogicalCalc(select=[a, IS TRUE(>(a, 0)) AS $f1, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -416,7 +416,7 @@ FlinkLogicalAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $2], EXPR$1=[MIN($1) ($0, 0))], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -427,7 +427,7 @@ FlinkLogicalCalc(select=[CASE(IS NOT NULL(EXPR$0), EXPR$0, 0) AS EXPR$0, EXPR$1] +- FlinkLogicalAggregate(group=[{2, 3}], EXPR$0=[COUNT($0) FILTER $1]) +- FlinkLogicalExpand(projects=[{a, $f1, b, 0 AS $e}, {a, $f1, null AS b, 1 AS $e}]) +- FlinkLogicalCalc(select=[a, IS TRUE(>(a, 0)) AS $f1, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -440,7 +440,7 @@ FlinkLogicalCalc(select=[CASE(IS NOT NULL(EXPR$0), EXPR$0, 0) AS EXPR$0, EXPR$1] LogicalProject(EXPR$0=[$2], EXPR$1=[$3]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}]], EXPR$0=[COUNT(DISTINCT $2)], EXPR$1=[SUM($0)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -452,7 +452,7 @@ FlinkLogicalCalc(select=[EXPR$0, EXPR$1]) +- FlinkLogicalExpand(projects=[{b, c, a, $e, b_0, 0 AS $e_0}, {b, c, null AS a, $e, b_0, 2 AS $e_0}]) +- FlinkLogicalExpand(projects=[{b, null AS c, a, 1 AS $e, b AS b_0}, {null AS b, c, a, 2 AS $e, b AS b_0}]) +- FlinkLogicalCalc(select=[b, c, a]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -465,7 +465,7 @@ FlinkLogicalCalc(select=[EXPR$0, EXPR$1]) LogicalProject(EXPR$0=[$2], EXPR$1=[$3]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}]], EXPR$0=[COUNT($2)], EXPR$1=[SUM(DISTINCT $3)]) +- LogicalProject(c=[$2], d=[$3], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -477,7 +477,7 @@ FlinkLogicalCalc(select=[CAST(EXPR$0 AS BIGINT) AS EXPR$0, EXPR$1]) +- FlinkLogicalExpand(projects=[{c, d, a, b, $e, 0 AS $e_0}, {c, d, a, null AS b, $e, 2 AS $e_0}]) +- FlinkLogicalExpand(projects=[{c, null AS d, a, b, 1 AS $e}, {null AS c, d, a, b, 2 AS $e}]) +- FlinkLogicalCalc(select=[c, d, a, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -489,7 +489,7 @@ FlinkLogicalCalc(select=[CAST(EXPR$0 AS BIGINT) AS EXPR$0, EXPR$1]) @@ -497,7 +497,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT(DISTINCT $0, $1)]) FlinkLogicalAggregate(group=[{}], EXPR$0=[COUNT($0, $1)]) +- FlinkLogicalAggregate(group=[{0, 1}]) +- FlinkLogicalCalc(select=[a, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -510,7 +510,7 @@ FlinkLogicalAggregate(group=[{}], EXPR$0=[COUNT($0, $1)]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}]], EXPR$0=[COUNT(DISTINCT $2, $3)]) +- LogicalProject(c=[$2], d=[$3], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -520,7 +520,7 @@ FlinkLogicalCalc(select=[EXPR$0]) +- FlinkLogicalAggregate(group=[{0, 1, 2, 3, 4}]) +- FlinkLogicalExpand(projects=[{c, null AS d, a, b, 1 AS $e}, {null AS c, d, a, b, 2 AS $e}]) +- FlinkLogicalCalc(select=[c, d, a, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -532,7 +532,7 @@ FlinkLogicalCalc(select=[EXPR$0]) @@ -543,7 +543,7 @@ FlinkLogicalCalc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2, EXPR$3]) +- FlinkLogicalAggregate(group=[{0, 1, 2}], EXPR$1=[COUNT()]) +- FlinkLogicalExpand(projects=[{a, b, 0 AS $e}, {a, null AS b, 1 AS $e}]) +- FlinkLogicalCalc(select=[a, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -555,7 +555,7 @@ FlinkLogicalCalc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2, EXPR$3]) @@ -566,7 +566,7 @@ FlinkLogicalCalc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2]) +- FlinkLogicalAggregate(group=[{0, 1, 2}], EXPR$1=[COUNT($0)]) +- FlinkLogicalExpand(projects=[{a, b, 0 AS $e}, {a, null AS b, 1 AS $e}]) +- FlinkLogicalCalc(select=[a, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -578,7 +578,7 @@ FlinkLogicalCalc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2]) @@ -589,7 +589,7 @@ FlinkLogicalCalc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2]) +- FlinkLogicalAggregate(group=[{0, 1, 2}], EXPR$1=[COUNT()]) +- FlinkLogicalExpand(projects=[{a, b, 0 AS $e}, {a, null AS b, 1 AS $e}]) +- FlinkLogicalCalc(select=[a, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -601,7 +601,7 @@ FlinkLogicalCalc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2]) ($0, 0))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -609,7 +609,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1) FILTER $2]) FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($2)]) +- FlinkLogicalAggregate(group=[{0, 1, 2}]) +- FlinkLogicalCalc(select=[d, IS TRUE(>(a, 0)) AS $f2, CASE(IS TRUE(>(a, 0)), c, null:VARCHAR(2147483647)) AS i$c]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e]) ]]> @@ -620,7 +620,7 @@ FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($2)]) @@ -630,7 +630,7 @@ FlinkLogicalCalc(select=[a, CAST(EXPR$1 AS BIGINT) AS EXPR$1, EXPR$2, EXPR$3]) +- FlinkLogicalCalc(select=[a, b, c, EXPR$1, =(CASE(=($e, 1), 1, =($e, 2), 2, 3), 1) AS $g_1, =(CASE(=($e, 1), 1, =($e, 2), 2, 3), 2) AS $g_2, =(CASE(=($e, 1), 1, =($e, 2), 2, 3), 3) AS $g_3]) +- FlinkLogicalAggregate(group=[{0, 1, 2, 3}], EXPR$1=[COUNT()]) +- FlinkLogicalExpand(projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.xml index 4d10c1f1f978d..1e905b99ba669 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.xml @@ -29,8 +29,8 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[SUM($3)]) +- LogicalProject(a2=[$0], b2=[$1], c2=[$2], a=[$3], b=[$4], c=[$5]) +- LogicalFilter(condition=[=($0, $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -38,9 +38,9 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[SUM($3)]) LogicalProject(a2=[$1], b2=[$0], c2=[$2], EXPR$3=[$3]) +- LogicalProject(b2=[$1], a2=[$0], c2=[$2], EXPR$3=[$4]) +- LogicalJoin(condition=[=($0, $3)], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalAggregate(group=[{0}], EXPR$3=[SUM($0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -57,8 +57,8 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[SUM($3)]) +- LogicalProject(a2=[$0], b2=[$1], c2=[$2], a=[$3], b=[$4], c=[$5]) +- LogicalFilter(condition=[=($0, $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -67,9 +67,9 @@ LogicalProject(a2=[$2], b2=[$0], c=[$1], EXPR$3=[$3]) +- LogicalProject(b2=[$1], c=[$3], a2=[$0], EXPR$3=[$4]) +- LogicalJoin(condition=[=($0, $2)], joinType=[inner]) :- LogicalProject(a2=[$0], b2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalAggregate(group=[{0, 2}], EXPR$3=[SUM($0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -86,8 +86,8 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[SUM($3)]) +- LogicalProject(a2=[$0], b2=[$1], c2=[$2], a=[$3], b=[$4], c=[$5]) +- LogicalFilter(condition=[=($1, $4)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -95,9 +95,9 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[SUM($3)]) LogicalProject(a2=[$1], b2=[$0], c2=[$2], EXPR$3=[$3]) +- LogicalProject(b2=[$1], a2=[$0], c2=[$2], EXPR$3=[$4]) +- LogicalJoin(condition=[=($1, $3)], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalAggregate(group=[{1}], EXPR$3=[SUM($0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -114,8 +114,8 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[SUM($3)]) +- LogicalProject(a2=[$0], b2=[$1], c2=[$2], a=[$3], b=[$4], c=[$5]) +- LogicalFilter(condition=[=($1, $4)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -124,9 +124,9 @@ LogicalProject(a2=[$2], b2=[$0], c=[$1], EXPR$3=[$3]) +- LogicalProject(b2=[$1], c=[$3], a2=[$0], EXPR$3=[$4]) +- LogicalJoin(condition=[=($1, $2)], joinType=[inner]) :- LogicalProject(a2=[$0], b2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalAggregate(group=[{1, 2}], EXPR$3=[SUM($0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -150,12 +150,12 @@ LogicalProject(EXPR$0=[$2], EXPR$1=[$3], EXPR$2=[$4], EXPR$3=[$5], a=[$0], b=[$1 :- LogicalJoin(condition=[true], joinType=[inner]) : :- LogicalAggregate(group=[{0}], b1=[COUNT($1)]) : : +- LogicalProject(a1=[$0], b=[$1]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, T]]) : +- LogicalProject(a2=[$1], b2=[$0]) : +- LogicalAggregate(group=[{0}], a2=[COUNT($1)]) : +- LogicalProject(b2=[$1], a=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -165,12 +165,12 @@ LogicalProject(EXPR$0=[$2], EXPR$1=[$3], EXPR$2=[$4], EXPR$3=[$5], a=[$0], b=[$1 +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) :- LogicalJoin(condition=[=($0, $3)], joinType=[inner]) : :- LogicalAggregate(group=[{0}], b1=[COUNT($1)]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, T]]) : +- LogicalProject(a2=[$1], b2=[$0]) : +- LogicalAggregate(group=[{1}], a2=[COUNT($0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T]]) +- LogicalAggregate(group=[{0, 1}], EXPR$6=[COUNT($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -189,8 +189,8 @@ LogicalProject(EXPR$0=[$2], EXPR$1=[$3], a=[$0], b=[$1], EXPR$4=[$4]) +- LogicalProject(a2=[$0], b2=[$1], c2=[$2], a=[$3], b=[$4], c=[$5]) +- LogicalFilter(condition=[=($1, $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -199,9 +199,9 @@ LogicalProject(EXPR$0=[$2], EXPR$1=[$3], a=[$0], b=[$1], EXPR$4=[$4]) +- LogicalProject(a=[$3], b=[$4], a2=[$1], b2=[$0], $f6=[*($2, $5)]) +- LogicalJoin(condition=[=($0, $3)], joinType=[inner]) :- LogicalProject(b2=[$1], a2=[$0], $f2=[CASE(IS NOT NULL($2), 1:BIGINT, 0:BIGINT)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalAggregate(group=[{0, 1}], agg#0=[COUNT()]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -216,8 +216,8 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT($0)]) +- LogicalJoin(condition=[=($0, $1)], joinType=[inner]) :- LogicalAggregate(group=[{0}]) : +- LogicalProject(a=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -227,9 +227,9 @@ LogicalAggregate(group=[{}], EXPR$0=[$SUM0($0)]) +- LogicalJoin(condition=[=($0, $2)], joinType=[inner]) :- LogicalProject(a=[$0], $f1=[CASE(IS NOT NULL($0), 1:BIGINT, 0:BIGINT)]) : +- LogicalAggregate(group=[{0}]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T]]) +- LogicalAggregate(group=[{0}], agg#0=[COUNT()]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -244,8 +244,8 @@ LogicalAggregate(group=[{}], EXPR$0=[SUM($0)]) +- LogicalJoin(condition=[=($0, $1)], joinType=[inner]) :- LogicalAggregate(group=[{0}]) : +- LogicalProject(a=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -254,9 +254,9 @@ LogicalAggregate(group=[{}], EXPR$0=[SUM($0)]) +- LogicalProject($f3=[CAST(*($0, $2)):INTEGER]) +- LogicalJoin(condition=[=($0, $1)], joinType=[inner]) :- LogicalAggregate(group=[{0}]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T]]) +- LogicalAggregate(group=[{0}], agg#0=[COUNT()]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateRemoveRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateRemoveRuleTest.xml index 1b160019f30c0..01c673857bd12 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateRemoveRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateRemoveRuleTest.xml @@ -24,14 +24,14 @@ limitations under the License. @@ -43,14 +43,14 @@ FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1)]) @@ -63,7 +63,7 @@ FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT()]) LogicalProject(a=[$0], s=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0, 2}, {0}, {1, 2}, {1}, {2}, {}]], s=[SUM($3)]) +- LogicalProject(a=[$0], c=[$2], d=[$3], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3]]) ]]> @@ -72,7 +72,7 @@ FlinkLogicalCalc(select=[a, s]) +- FlinkLogicalAggregate(group=[{0, 1, 2, 4}], s=[SUM($3)]) +- FlinkLogicalExpand(projects=[{a, c, d, b, 0 AS $e}, {a, c, null AS d, b, 1 AS $e}, {a, null AS c, d, b, 2 AS $e}, {a, null AS c, null AS d, b, 3 AS $e}, {null AS a, c, d, b, 4 AS $e}, {null AS a, c, null AS d, b, 5 AS $e}, {null AS a, null AS c, d, b, 6 AS $e}, {null AS a, null AS c, null AS d, b, 7 AS $e}]) +- FlinkLogicalCalc(select=[a, c, d, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable3]], fields=[a, b, c, d]) ]]> @@ -84,14 +84,14 @@ FlinkLogicalCalc(select=[a, s]) @@ -104,7 +104,7 @@ FlinkLogicalAggregate(group=[{0}], EXPR$1=[MAX($1)]) LogicalProject(a=[$0], s=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1}, {0, 2}]], s=[SUM($3)]) +- LogicalProject(a=[$0], c=[$2], d=[$3], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3]]) ]]> @@ -112,7 +112,7 @@ LogicalProject(a=[$0], s=[$3]) FlinkLogicalCalc(select=[a, b AS s]) +- FlinkLogicalExpand(projects=[{a, c, null AS d, b, 1 AS $e}, {a, null AS c, d, b, 2 AS $e}]) +- FlinkLogicalCalc(select=[a, c, d, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable3]], fields=[a, b, c, d]) ]]> @@ -125,7 +125,7 @@ FlinkLogicalCalc(select=[a, b AS s]) LogicalProject(a=[$0], s=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], s=[SUM($2)]) +- LogicalProject(a=[$0], c=[$2], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3]]) ]]> @@ -134,7 +134,7 @@ FlinkLogicalCalc(select=[a, s]) +- FlinkLogicalAggregate(group=[{0, 1, 3}], s=[SUM($2)]) +- FlinkLogicalExpand(projects=[{a, c, b, 0 AS $e}, {a, null AS c, b, 1 AS $e}, {null AS a, null AS c, b, 3 AS $e}]) +- FlinkLogicalCalc(select=[a, c, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable3]], fields=[a, b, c, d]) ]]> @@ -146,14 +146,14 @@ FlinkLogicalCalc(select=[a, s]) @@ -166,7 +166,7 @@ FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1)], EXPR$2=[SUM($1)]) LogicalProject(a=[$0], EXPR$1=[$2], EXPR$2=[$3]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[COUNT(DISTINCT $2)], EXPR$2=[SUM(DISTINCT $3)]) +- LogicalProject(a=[$0], d=[$3], c=[$2], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3]]) ]]> @@ -176,7 +176,7 @@ FlinkLogicalCalc(select=[a, EXPR$1, EXPR$2]) +- FlinkLogicalCalc(select=[a, d, c, b, =(CASE(=($e, 1), 1, 2), 1) AS $g_1, =(CASE(=($e, 1), 1, 2), 2) AS $g_2]) +- FlinkLogicalExpand(projects=[{a, d, c, null AS b, 1 AS $e}, {a, d, null AS c, b, 2 AS $e}]) +- FlinkLogicalCalc(select=[a, d, c, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable3]], fields=[a, b, c, d]) ]]> @@ -187,13 +187,13 @@ FlinkLogicalCalc(select=[a, EXPR$1, EXPR$2]) @@ -206,7 +206,7 @@ FlinkLogicalCalc(select=[a, b AS EXPR$1, b AS EXPR$2, c AS EXPR$3]) LogicalProject(a=[$0], EXPR$1=[$2], EXPR$2=[$3]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[COUNT(DISTINCT $2)], EXPR$2=[SUM($3)]) +- LogicalProject(a=[$0], d=[$3], c=[$2], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3]]) ]]> @@ -216,7 +216,7 @@ FlinkLogicalCalc(select=[a, EXPR$1, EXPR$2]) +- FlinkLogicalCalc(select=[a, d, c, b AS EXPR$2, =(CASE(=($e, 0), 0, 1), 0) AS $g_0, =(CASE(=($e, 0), 0, 1), 1) AS $g_1]) +- FlinkLogicalExpand(projects=[{a, d, c, b, 0 AS $e}, {a, d, null AS c, b, 1 AS $e}]) +- FlinkLogicalCalc(select=[a, d, c, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable3]], fields=[a, b, c, d]) ]]> @@ -229,7 +229,7 @@ FlinkLogicalCalc(select=[a, EXPR$1, EXPR$2]) LogicalProject(a=[$0], s=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0}, {}]], s=[SUM($3)]) +- LogicalProject(a=[$0], c=[$2], d=[$3], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3]]) ]]> @@ -238,7 +238,7 @@ FlinkLogicalCalc(select=[a, s]) +- FlinkLogicalAggregate(group=[{0, 1, 2, 4}], s=[SUM($3)]) +- FlinkLogicalExpand(projects=[{a, c, d, b, 0 AS $e}, {a, c, null AS d, b, 1 AS $e}, {a, null AS c, null AS d, b, 3 AS $e}, {null AS a, null AS c, null AS d, b, 7 AS $e}]) +- FlinkLogicalCalc(select=[a, c, d, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable3]], fields=[a, b, c, d]) ]]> @@ -250,14 +250,14 @@ FlinkLogicalCalc(select=[a, s]) @@ -269,14 +269,14 @@ FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1)]) @@ -288,7 +288,7 @@ FlinkLogicalCalc(select=[a, EXPR$1]) @@ -297,7 +297,7 @@ FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $3], EXPR$2=[MIN($2) +- FlinkLogicalCalc(select=[a, b, b_0 AS EXPR$2, =(CASE(=($e, 0), 0, 1), 0) AS $g_0, =(CASE(=($e, 0), 0, 1), 1) AS $g_1]) +- FlinkLogicalExpand(projects=[{a, b, 0 AS $e, b AS b_0}, {a, null AS b, 1 AS $e, b AS b_0}]) +- FlinkLogicalCalc(select=[a, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c]) ]]> @@ -310,7 +310,7 @@ FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $3], EXPR$2=[MIN($2) LogicalProject(a=[$0], EXPR$1=[$2], EXPR$2=[$3]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[COUNT(DISTINCT $2)], EXPR$2=[SUM($2)]) +- LogicalProject(a=[$0], c=[$2], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -320,7 +320,7 @@ FlinkLogicalCalc(select=[a, EXPR$1, EXPR$2]) +- FlinkLogicalCalc(select=[a, c, b, b_0 AS EXPR$2, =(CASE(=($e, 0), 0, 1), 0) AS $g_0, =(CASE(=($e, 0), 0, 1), 1) AS $g_1]) +- FlinkLogicalExpand(projects=[{a, c, b, 0 AS $e, b AS b_0}, {a, c, null AS b, 1 AS $e, b AS b_0}]) +- FlinkLogicalCalc(select=[a, c, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c]) ]]> @@ -332,7 +332,7 @@ FlinkLogicalCalc(select=[a, EXPR$1, EXPR$2]) @@ -341,7 +341,7 @@ FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $3], EXPR$2=[MIN($2) +- FlinkLogicalCalc(select=[a, c, b AS EXPR$2, =(CASE(=($e, 0), 0, 1), 0) AS $g_0, =(CASE(=($e, 0), 0, 1), 1) AS $g_1]) +- FlinkLogicalExpand(projects=[{a, c, b, 0 AS $e}, {a, null AS c, b, 1 AS $e}]) +- FlinkLogicalCalc(select=[a, c, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable3]], fields=[a, b, c, d]) ]]> @@ -354,7 +354,7 @@ FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $3], EXPR$2=[MIN($2) LogicalProject(a=[$0], EXPR$1=[$2], EXPR$2=[$3]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[COUNT(DISTINCT $2)], EXPR$2=[SUM($3)]) +- LogicalProject(a=[$0], d=[$3], c=[$2], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3]]) ]]> @@ -364,7 +364,7 @@ FlinkLogicalCalc(select=[a, EXPR$1, EXPR$2]) +- FlinkLogicalCalc(select=[a, d, c, b AS EXPR$2, =(CASE(=($e, 0), 0, 1), 0) AS $g_0, =(CASE(=($e, 0), 0, 1), 1) AS $g_1]) +- FlinkLogicalExpand(projects=[{a, d, c, b, 0 AS $e}, {a, d, null AS c, b, 1 AS $e}]) +- FlinkLogicalCalc(select=[a, d, c, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable3]], fields=[a, b, c, d]) ]]> @@ -376,14 +376,14 @@ FlinkLogicalCalc(select=[a, EXPR$1, EXPR$2]) ($1, 0))], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> (b, 0)) AS $f2, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c]) ]]> @@ -394,13 +394,13 @@ FlinkLogicalAggregate(group=[{0}], EXPR$1=[MIN($1) FILTER $2], EXPR$2=[MAX($3)]) @@ -412,13 +412,13 @@ FlinkLogicalCalc(select=[a, b]) @@ -430,13 +430,13 @@ FlinkLogicalCalc(select=[a, +(b, 1) AS EXPR$1, c, b AS s]) @@ -447,13 +447,13 @@ FlinkLogicalCalc(select=[a, b]) @@ -485,15 +485,15 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalAggregate(group=[{}], EXPR$0=[SUM($0)]) LogicalProject(a=[$0]) LogicalFilter(condition=[=(1, 2)]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -506,7 +506,7 @@ FlinkLogicalCalc(select=[a, b, c]) LogicalAggregate(group=[{}], EXPR$0=[SUM($0)]) +- LogicalProject(a=[$0]) +- LogicalFilter(condition=[=(1, 2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRuleTest.xml index ec3cb6dbb60df..977a321875633 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRuleTest.xml @@ -25,14 +25,14 @@ limitations under the License. LogicalProject(a2=[random_udf($0)], a3=[random_udf($0)]) +- LogicalFilter(condition=[>($1, 10)]) +- LogicalProject(a1=[random_udf($0)], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> (b, 10)]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -45,13 +45,13 @@ FlinkLogicalCalc(select=[random_udf(a1) AS a2, random_udf(a1) AS a3]) LogicalProject(a=[$0], c=[$2]) +- LogicalFilter(condition=[>($1, 10)]) +- LogicalProject(a=[$0], b1=[random_udf($1)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> (random_udf(b), 10)]) -+- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -64,13 +64,13 @@ FlinkLogicalCalc(select=[a, c], where=[>(random_udf(b), 10)]) LogicalProject(a1=[$0], b2=[$1]) +- LogicalFilter(condition=[>(CAST($2):BIGINT, 10)]) +- LogicalProject(a1=[random_udf($0)], b2=[random_udf($1)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> (CAST(c AS BIGINT), 10)]) -+- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -83,13 +83,13 @@ FlinkLogicalCalc(select=[random_udf(a) AS a1, random_udf(b) AS b2], where=[>(CAS LogicalProject(c=[$2]) +- LogicalFilter(condition=[>($0, $1)]) +- LogicalProject(a1=[random_udf($0)], b2=[random_udf($1)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> (random_udf(a), random_udf(b))]) -+- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -102,13 +102,13 @@ FlinkLogicalCalc(select=[c], where=[>(random_udf(a), random_udf(b))]) LogicalProject(a2=[random_udf($0)]) +- LogicalFilter(condition=[>($1, 10)]) +- LogicalProject(a1=[random_udf($0)], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> (b, 10)]) -+- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -121,13 +121,13 @@ FlinkLogicalCalc(select=[random_udf(random_udf(a)) AS a2], where=[>(b, 10)]) LogicalProject(a=[$0]) +- LogicalFilter(condition=[>(random_udf($0), 10)]) +- LogicalProject(a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> (random_udf(a), 10)]) -+- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -140,13 +140,13 @@ FlinkLogicalCalc(select=[a], where=[>(random_udf(a), 10)]) LogicalProject(a=[$0], c=[$2]) +- LogicalFilter(condition=[>(CAST($2):BIGINT, 10)]) +- LogicalProject(a=[$0], a1=[random_udf($0)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> (CAST(c AS BIGINT), 10)]) -+- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -160,13 +160,13 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[=($1, $0)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[=($0, $1)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -179,14 +179,14 @@ FlinkLogicalCalc(select=[a, b], where=[=(a, b)]) LogicalProject(a=[$0], a1=[$1]) +- LogicalFilter(condition=[>($1, 10)]) +- LogicalProject(a=[$0], a1=[random_udf($0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> (a1, 10)]) +- FlinkLogicalCalc(select=[a, random_udf(a) AS a1]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRuleTest.xml index b935a80648e17..2cb129a37754c 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRuleTest.xml @@ -26,9 +26,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[=($0, $4)]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[=($0, $2)], joinType=[full]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -37,9 +37,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[=($0, $4)]) +- MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[INNER, INNER]], outerJoinConditions=[[NULL, NULL]], projFields=[[ALL, ALL]]) :- MultiJoin(joinFilter=[=($0, $2)], isFullOuterJoin=[true], joinTypes=[[INNER, INNER]], outerJoinConditions=[[NULL, NULL]], projFields=[[ALL, ALL]]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -52,18 +52,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) :- LogicalJoin(condition=[=($0, $2)], joinType=[full]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -76,18 +76,18 @@ MultiJoin(joinFilter=[=($0, $4)], isFullOuterJoin=[false], joinTypes=[[INNER, IN LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($0, $4)], joinType=[left]) :- LogicalJoin(condition=[=($0, $2)], joinType=[full]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -100,18 +100,18 @@ MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[INNER, LEFT]], LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($0, $4)], joinType=[right]) :- LogicalJoin(condition=[=($0, $2)], joinType=[full]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -124,11 +124,11 @@ MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[RIGHT, INNER]] LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalFilter(condition=[IN($0, { LogicalProject(e=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T3]]) })]) +- LogicalJoin(condition=[=($0, $2)], joinType=[full]) - :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -136,10 +136,10 @@ LogicalProject(e=[$0]) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalJoin(condition=[=($0, $4)], joinType=[semi]) :- MultiJoin(joinFilter=[=($0, $2)], isFullOuterJoin=[true], joinTypes=[[INNER, INNER]], outerJoinConditions=[[NULL, NULL]], projFields=[[ALL, ALL]]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalProject(e=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -155,12 +155,12 @@ WHERE NOT EXISTS (SELECT e FROM T3 WHERE a = e) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T3]]) }))], variablesSet=[[$cor0]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalJoin(condition=[=($0, $2)], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -168,11 +168,11 @@ LogicalFilter(condition=[=($cor0.a, $0)]) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalJoin(condition=[=($0, $4)], joinType=[anti]) :- MultiJoin(joinFilter=[=($0, $2)], isFullOuterJoin=[false], joinTypes=[[INNER, INNER]], outerJoinConditions=[[NULL, NULL]], projFields=[[{0, 1}, {0, 1}]]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalProject(e=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -186,9 +186,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $2), =($0, $4))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -196,9 +196,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $2), =($0, $4))]) +- MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[INNER, INNER, INNER]], outerJoinConditions=[[NULL, NULL, NULL]], projFields=[[ALL, ALL, ALL]]) - :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - :- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -214,12 +214,12 @@ WHERE NOT EXISTS (SELECT e FROM T3 WHERE a = e) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T3]]) }))], variablesSet=[[$cor0]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalJoin(condition=[=($0, $2)], joinType=[right]) - :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -227,11 +227,11 @@ LogicalFilter(condition=[=($cor0.a, $0)]) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalJoin(condition=[=($0, $4)], joinType=[anti]) :- MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[RIGHT, INNER]], outerJoinConditions=[[=($0, $2), NULL]], projFields=[[{0, 1}, {0, 1}]]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalProject(e=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -244,17 +244,17 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($0, $4)], joinType=[left]) :- LogicalJoin(condition=[=($0, $2)], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -274,21 +274,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], g=[$6], h=[$7], i :- LogicalJoin(condition=[=($0, $6)], joinType=[inner]) : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) : : :- LogicalJoin(condition=[=($0, $2)], joinType=[inner]) - : : : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(g, h)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(i, j)]]]) + : : : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T4]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T5]]) ]]> @@ -301,18 +301,18 @@ MultiJoin(joinFilter=[AND(=($0, $6), =($0, $2))], isFullOuterJoin=[false], joinT LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($0, $4)], joinType=[right]) :- LogicalJoin(condition=[=($0, $2)], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -332,11 +332,11 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], g=[$6], h=[$7], i :- LogicalJoin(condition=[=($0, $6)], joinType=[inner]) : :- LogicalJoin(condition=[=($0, $4)], joinType=[right]) : : :- LogicalJoin(condition=[=($0, $2)], joinType=[inner]) - : : : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(g, h)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(i, j)]]]) + : : : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T4]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T5]]) ]]> @@ -345,11 +345,11 @@ MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[RIGHT, INNER]] :- MultiJoin(joinFilter=[=($0, $6)], isFullOuterJoin=[false], joinTypes=[[INNER, INNER]], outerJoinConditions=[[NULL, NULL]], projFields=[[ALL, ALL]]) : :- MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[RIGHT, INNER]], outerJoinConditions=[[=($0, $4), NULL]], projFields=[[ALL, ALL]]) : : :- MultiJoin(joinFilter=[=($0, $2)], isFullOuterJoin=[false], joinTypes=[[INNER, INNER]], outerJoinConditions=[[NULL, NULL]], projFields=[[ALL, ALL]]) -: : : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) -: : : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) -: : +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) -: +- LogicalTableScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(g, h)]]]) -+- LogicalTableScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(i, j)]]]) +: : : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +: : : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +: : +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, T4]]) ++- LogicalTableScan(table=[[default_catalog, default_database, T5]]) ]]> @@ -362,12 +362,12 @@ MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[RIGHT, INNER]] LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalFilter(condition=[IN($0, { LogicalProject(e=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T3]]) })]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalJoin(condition=[=($0, $2)], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -375,10 +375,10 @@ LogicalProject(e=[$0]) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalJoin(condition=[=($0, $4)], joinType=[semi]) :- MultiJoin(joinFilter=[=($0, $2)], isFullOuterJoin=[false], joinTypes=[[INNER, INNER]], outerJoinConditions=[[NULL, NULL]], projFields=[[{0, 1}, {0, 1}]]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalProject(e=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -394,12 +394,12 @@ WHERE NOT EXISTS (SELECT e FROM T3 WHERE a = e) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T3]]) }))], variablesSet=[[$cor0]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalJoin(condition=[=($0, $2)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -407,11 +407,11 @@ LogicalFilter(condition=[=($cor0.a, $0)]) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalJoin(condition=[=($0, $4)], joinType=[anti]) :- MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[INNER, LEFT]], outerJoinConditions=[[NULL, =($0, $2)]], projFields=[[{0, 1}, {0, 1}]]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalProject(e=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -431,21 +431,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], g=[$6], h=[$7], i :- LogicalJoin(condition=[=($0, $6)], joinType=[left]) : :- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) : : :- LogicalJoin(condition=[=($0, $2)], joinType=[left]) - : : : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(g, h)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(i, j)]]]) + : : : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T4]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T5]]) ]]> @@ -458,17 +458,17 @@ MultiJoin(joinFilter=[AND(=($0, $8), =($0, $4))], isFullOuterJoin=[false], joinT LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($0, $4)], joinType=[left]) :- LogicalJoin(condition=[=($0, $2)], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -481,18 +481,18 @@ MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[INNER, LEFT, L LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($0, $4)], joinType=[right]) :- LogicalJoin(condition=[=($0, $2)], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -505,11 +505,11 @@ MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[RIGHT, INNER]] LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalFilter(condition=[IN($0, { LogicalProject(e=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T3]]) })]) +- LogicalJoin(condition=[=($0, $2)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -517,10 +517,10 @@ LogicalProject(e=[$0]) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalJoin(condition=[=($0, $4)], joinType=[semi]) :- MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[INNER, LEFT]], outerJoinConditions=[[NULL, =($0, $2)]], projFields=[[ALL, ALL]]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalProject(e=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -541,21 +541,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], g=[$6], h=[$7], i :- LogicalJoin(condition=[=($0, $6)], joinType=[left]) : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) : : :- LogicalJoin(condition=[=($0, $2)], joinType=[left]) - : : : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(g, h)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(i, j)]]]) + : : : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T4]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T5]]) ]]> @@ -568,18 +568,18 @@ MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[INNER, LEFT, L LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($0, $4)], joinType=[right]) :- LogicalJoin(condition=[=($0, $2)], joinType=[right]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -592,17 +592,17 @@ MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[RIGHT, INNER]] LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) :- LogicalJoin(condition=[=($0, $2)], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -622,11 +622,11 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], g=[$6], h=[$7], i :- LogicalJoin(condition=[=($0, $6)], joinType=[right]) : :- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) : : :- LogicalJoin(condition=[=($0, $2)], joinType=[right]) - : : : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(g, h)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(i, j)]]]) + : : : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T4]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T5]]) ]]> @@ -635,11 +635,11 @@ MultiJoin(joinFilter=[=($0, $8)], isFullOuterJoin=[false], joinTypes=[[INNER, IN :- MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[RIGHT, INNER]], outerJoinConditions=[[=($0, $6), NULL]], projFields=[[ALL, ALL]]) : :- MultiJoin(joinFilter=[=($0, $4)], isFullOuterJoin=[false], joinTypes=[[INNER, INNER]], outerJoinConditions=[[NULL, NULL]], projFields=[[ALL, ALL]]) : : :- MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[RIGHT, INNER]], outerJoinConditions=[[=($0, $2), NULL]], projFields=[[ALL, ALL]]) -: : : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) -: : : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) -: : +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) -: +- LogicalTableScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(g, h)]]]) -+- LogicalTableScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(i, j)]]]) +: : : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +: : : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +: : +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, T4]]) ++- LogicalTableScan(table=[[default_catalog, default_database, T5]]) ]]> @@ -652,18 +652,18 @@ MultiJoin(joinFilter=[=($0, $8)], isFullOuterJoin=[false], joinTypes=[[INNER, IN LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) :- LogicalJoin(condition=[=($0, $2)], joinType=[right]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -676,18 +676,18 @@ MultiJoin(joinFilter=[=($0, $4)], isFullOuterJoin=[false], joinTypes=[[INNER, IN LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($0, $4)], joinType=[left]) :- LogicalJoin(condition=[=($0, $2)], joinType=[right]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -700,17 +700,17 @@ MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[INNER, LEFT]], LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($2, $4)], joinType=[inner]) :- LogicalJoin(condition=[=($0, $2)], joinType=[right]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -723,17 +723,17 @@ MultiJoin(joinFilter=[=($2, $4)], isFullOuterJoin=[false], joinTypes=[[RIGHT, IN LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($2, $4)], joinType=[left]) :- LogicalJoin(condition=[=($0, $2)], joinType=[right]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -746,11 +746,11 @@ MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[RIGHT, INNER, LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalFilter(condition=[IN($0, { LogicalProject(e=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T3]]) })]) +- LogicalJoin(condition=[=($0, $2)], joinType=[right]) - :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -758,10 +758,10 @@ LogicalProject(e=[$0]) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalJoin(condition=[=($0, $4)], joinType=[semi]) :- MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[RIGHT, INNER]], outerJoinConditions=[[=($0, $2), NULL]], projFields=[[ALL, ALL]]) - : :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(c, d)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalProject(e=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -773,20 +773,20 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) @@ -798,19 +798,19 @@ MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[RIGHT, INNER]] diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLimit0RemoveRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLimit0RemoveRuleTest.xml index d4354855679d8..54cf953134889 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLimit0RemoveRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLimit0RemoveRuleTest.xml @@ -26,9 +26,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalSort(fetch=[0]) LogicalProject(a=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -48,16 +48,16 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalSort(fetch=[0]) LogicalProject(a=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -72,16 +72,16 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalSort(fetch=[0]) LogicalProject(a=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -94,17 +94,17 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) @@ -119,15 +119,15 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalSort(fetch=[0]) LogicalProject(a=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -139,7 +139,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) @@ -156,7 +156,7 @@ LogicalValues(tuples=[[]]) @@ -174,7 +174,7 @@ LogicalValues(tuples=[[]]) LogicalProject(a=[$0]) +- LogicalSort(fetch=[0]) +- LogicalProject(a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -192,7 +192,7 @@ LogicalProject(a=[$0]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForConstantRangeTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForConstantRangeTest.xml index e0aadb6eb9ea3..29c9c82b6f8d3 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForConstantRangeTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForConstantRangeTest.xml @@ -29,14 +29,14 @@ WHERE rk < 10 LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[<($2, 10)]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, rk)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -55,7 +55,7 @@ WHERE rk < 10 LogicalProject(a=[$0], b=[$1], rk=[$2], rn=[$3]) +- LogicalFilter(condition=[<($2, 10)]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)], rn=[ROW_NUMBER() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -63,7 +63,7 @@ LogicalProject(a=[$0], b=[$1], rk=[$2], rn=[$3]) FlinkLogicalCalc(select=[a, b, w0$o0, w1$o0], where=[<(w0$o0, 10)]) +- FlinkLogicalOverAggregate(window#0=[window(partition {1} order by [0 ASC-nulls-first] aggs [RANK()])], window#1=[window(partition {1} order by [0 ASC-nulls-first] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) +- FlinkLogicalCalc(select=[a, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -82,14 +82,14 @@ WHERE rk1 < 10 LogicalProject(a=[$0], b=[$1], rk1=[$2], rk2=[$3]) +- LogicalFilter(condition=[<($2, 10)]) +- LogicalProject(a=[$0], b=[$1], rk1=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)], rk2=[RANK() OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -108,7 +108,7 @@ WHERE rk1 < 10 LogicalProject(a=[$0], b=[$1], rk1=[$2], rk2=[$3]) +- LogicalFilter(condition=[<($2, 10)]) +- LogicalProject(a=[$0], b=[$1], rk1=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)], rk2=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -116,7 +116,7 @@ LogicalProject(a=[$0], b=[$1], rk1=[$2], rk2=[$3]) FlinkLogicalCalc(select=[a, b, w0$o0 AS rk1, w0$o0 AS rk2]) +- FlinkLogicalRank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=9], partitionBy=[b], orderBy=[a ASC], select=[a, b, w0$o0]) +- FlinkLogicalCalc(select=[a, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -133,14 +133,14 @@ WHERE rk = 2 LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[=($2, 2)]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST, $2 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -157,14 +157,14 @@ WHERE rk > 2 LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[>($2, 2)]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST, $2 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> (w0$o0, 2)]) +- FlinkLogicalOverAggregate(window#0=[window(partition {1} order by [0 ASC-nulls-first, 2 ASC-nulls-first] aggs [RANK()])]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -181,14 +181,14 @@ WHERE rk <= 2 AND rk > -2 LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[AND(<=($2, 2), >($2, -2))]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1, $2 ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -205,7 +205,7 @@ WHERE rk <= 2 AND a > 10 LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[AND(<=($2, 2), >($0, 10))]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -213,7 +213,7 @@ LogicalProject(a=[$0], b=[$1], rk=[$2]) FlinkLogicalCalc(select=[a, b, w0$o0], where=[>(a, 10)]) +- FlinkLogicalRank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=2], partitionBy=[b], orderBy=[a ASC], select=[a, b, w0$o0]) +- FlinkLogicalCalc(select=[a, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -230,7 +230,7 @@ WHERE rn <= 2 LogicalProject(a=[$0], b=[$1], rn=[$2]) +- LogicalFilter(condition=[<=($2, 2)]) +- LogicalProject(a=[$0], b=[$1], rn=[ROW_NUMBER() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -238,7 +238,7 @@ LogicalProject(a=[$0], b=[$1], rn=[$2]) FlinkLogicalCalc(select=[a, b, w0$o0], where=[<=(w0$o0, 2)]) +- FlinkLogicalOverAggregate(window#0=[window(partition {1} order by [0 ASC-nulls-first] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) +- FlinkLogicalCalc(select=[a, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -255,14 +255,14 @@ WHERE rk < a LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[<($2, $0)]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -279,14 +279,14 @@ WHERE rk > a LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[>($2, $0)]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> (w0$o0, a)]) +- FlinkLogicalOverAggregate(window#0=[window(partition {1} order by [2 ASC-nulls-first] aggs [RANK()])]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -303,14 +303,14 @@ WHERE rk < a and b > 5 LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[AND(<($2, $0), >(CAST($1):BIGINT, 5))]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> (CAST(b AS BIGINT), 5))]) +- FlinkLogicalOverAggregate(window#0=[window(partition {1} order by [2 ASC-nulls-first] aggs [RANK()])]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -327,14 +327,14 @@ WHERE rk = b LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[=($2, $1)]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $0 ORDER BY $2 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -351,14 +351,14 @@ WHERE rk < 10 LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[<($2, 10)]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -369,14 +369,14 @@ FlinkLogicalRank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=9], partitionB diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForRangeEndTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForRangeEndTest.xml index ccadc62ec26a1..15feb7852defe 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForRangeEndTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForRangeEndTest.xml @@ -31,14 +31,14 @@ WHERE rk < 10 LogicalProject(a=[$0], b=[$1], rk=[$2], rn=[$3]) +- LogicalFilter(condition=[<($2, 10)]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)], rn=[ROW_NUMBER() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -57,14 +57,14 @@ WHERE rk1 < 10 LogicalProject(a=[$0], b=[$1], rk1=[$2], rk2=[$3]) +- LogicalFilter(condition=[<($2, 10)]) +- LogicalProject(a=[$0], b=[$1], rk1=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)], rk2=[RANK() OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -83,7 +83,7 @@ WHERE rk1 < 10 LogicalProject(a=[$0], b=[$1], rk1=[$2], rk2=[$3]) +- LogicalFilter(condition=[<($2, 10)]) +- LogicalProject(a=[$0], b=[$1], rk1=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)], rk2=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -91,7 +91,7 @@ LogicalProject(a=[$0], b=[$1], rk1=[$2], rk2=[$3]) FlinkLogicalCalc(select=[a, b, rk2 AS rk1, rk2]) +- FlinkLogicalRank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=9], partitionBy=[b], orderBy=[a ASC], select=[a, b, rk2]) +- FlinkLogicalCalc(select=[a, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -108,14 +108,14 @@ WHERE rk = 2 LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[=($2, 2)]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST, $2 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -132,14 +132,14 @@ WHERE rk <= 2 AND rk > -2 LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[AND(<=($2, 2), >($2, -2))]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1, $2 ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -156,7 +156,7 @@ WHERE rk <= 2 AND a > 10 LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[AND(<=($2, 2), >($0, 10))]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -164,7 +164,7 @@ LogicalProject(a=[$0], b=[$1], rk=[$2]) FlinkLogicalCalc(select=[a, b, w0$o0], where=[>(a, 10)]) +- FlinkLogicalRank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=2], partitionBy=[b], orderBy=[a ASC], select=[a, b, w0$o0]) +- FlinkLogicalCalc(select=[a, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -181,14 +181,14 @@ WHERE rn <= 2 LogicalProject(a=[$0], b=[$1], rn=[$2]) +- LogicalFilter(condition=[<=($2, 2)]) +- LogicalProject(a=[$0], b=[$1], rn=[ROW_NUMBER() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -205,14 +205,14 @@ WHERE rk < a LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[<($2, $0)]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -229,14 +229,14 @@ WHERE rk > a LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[>($2, $0)]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> (w0$o0, a)]) +- FlinkLogicalOverAggregate(window#0=[window(partition {1} order by [2 ASC-nulls-first] aggs [RANK()])]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -253,14 +253,14 @@ WHERE rk < a and b > 5 LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[AND(<($2, $0), >(CAST($1):BIGINT, 5))]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> (CAST(b AS BIGINT), 5)]) +- FlinkLogicalRank(rankType=[RANK], rankRange=[rankEnd=a], partitionBy=[b], orderBy=[c ASC], select=[a, b, c, w0$o0]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -277,14 +277,14 @@ WHERE rk = b LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[=($2, $1)]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $0 ORDER BY $2 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -301,14 +301,14 @@ WHERE rk < 10 LogicalProject(a=[$0], b=[$1], rk=[$2]) +- LogicalFilter(condition=[<($2, 10)]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (ORDER BY $0 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -319,14 +319,14 @@ FlinkLogicalRank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=9], partitionB diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkPruneEmptyRulesTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkPruneEmptyRulesTest.xml index f9c85b09ab147..de3b51cfba2c6 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkPruneEmptyRulesTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkPruneEmptyRulesTest.xml @@ -26,9 +26,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=(1, 0)]) - LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T2]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) ]]> @@ -48,15 +48,15 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=(1, 0)]) - LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T2]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinFilterTransposeRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinFilterTransposeRuleTest.xml index 39dd1e872d87e..2b91cde27c08a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinFilterTransposeRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinFilterTransposeRuleTest.xml @@ -28,11 +28,11 @@ SELECT * FROM (SELECT * FROM MyTable1 WHERE a > 10) t LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($1, { LogicalProject(e=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) }))]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -40,9 +40,9 @@ LogicalProject(e=[$1]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>($0, 10)]) +- LogicalJoin(condition=[OR(IS NULL($1), IS NULL($3), =($1, $3))], joinType=[anti]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -58,11 +58,11 @@ SELECT * FROM (SELECT * FROM MyTable1 WHERE a > 10) t LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($1, { LogicalProject(e=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) })]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -70,9 +70,9 @@ LogicalProject(e=[$1]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>($0, 10)]) +- LogicalJoin(condition=[=($1, $3)], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinJoinTransposeRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinJoinTransposeRuleTest.xml index 2cc3155a2b25d..05f82738fb672 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinJoinTransposeRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinJoinTransposeRuleTest.xml @@ -25,11 +25,11 @@ limitations under the License. LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), EXISTS({ LogicalFilter(condition=[<($1, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -38,13 +38,13 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[$6], joinType=[semi]) :- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject($f0=[IS NOT NULL($0)]) +- LogicalAggregate(group=[{}], m=[MIN($0)]) +- LogicalProject(i=[true]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -57,11 +57,11 @@ LogicalProject(a=[$0], f=[$5]) LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), EXISTS({ LogicalFilter(condition=[AND(=($0, $cor0.a), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -70,11 +70,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[=($3, $0)], joinType=[semi]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(i=[$0]) : +- LogicalFilter(condition=[<($1, 50)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -87,11 +87,11 @@ LogicalProject(a=[$0], f=[$5]) LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), EXISTS({ LogicalFilter(condition=[AND(=($0, $cor0.d), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -99,12 +99,12 @@ LogicalFilter(condition=[AND(=($0, $cor0.d), <($1, 50))]) LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalJoin(condition=[=($3, $0)], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -117,11 +117,11 @@ LogicalProject(a=[$0], f=[$5]) LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), EXISTS({ LogicalFilter(condition=[AND(>($0, $cor0.a), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -130,11 +130,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[>($3, $0)], joinType=[semi]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(i=[$0]) : +- LogicalFilter(condition=[<($1, 50)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -147,11 +147,11 @@ LogicalProject(a=[$0], f=[$5]) LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), EXISTS({ LogicalFilter(condition=[AND(<($0, $cor0.d), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -159,12 +159,12 @@ LogicalFilter(condition=[AND(<($0, $cor0.d), <($1, 50))]) LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalJoin(condition=[<($3, $0)], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -178,11 +178,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), IN($3, { LogicalProject(i=[$0]) LogicalFilter(condition=[<($1, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -190,12 +190,12 @@ LogicalProject(i=[$0]) LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalJoin(condition=[=($0, $3)], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -209,11 +209,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), IN($3, { LogicalProject(i=[$0]) LogicalFilter(condition=[AND(=($cor0.e, $1), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -221,12 +221,12 @@ LogicalProject(i=[$0]) LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalJoin(condition=[AND(=($0, $3), =($1, $4))], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0], j=[$1]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -240,11 +240,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[<($1, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -253,11 +253,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[=($0, $3)], joinType=[semi]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(i=[$0]) : +- LogicalFilter(condition=[<($1, 50)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -271,11 +271,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -284,11 +284,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[AND(=($0, $3), =($1, $4))], joinType=[semi]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(i=[$0], j=[$1]) : +- LogicalFilter(condition=[<($1, 50)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -302,11 +302,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), IN($0, $4, { LogicalProject(i=[$0], j=[$1]) LogicalFilter(condition=[<($1, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -315,11 +315,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[AND(=($0, $6), =($4, $7))], joinType=[semi]) :- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0], j=[$1]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -333,11 +333,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[AND(=($cor0.e, $1), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -346,11 +346,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[AND(=($0, $6), =($4, $7))], joinType=[semi]) :- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0], j=[$1]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -364,12 +364,12 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(>($4, 100), IN($3, { LogicalProject(i=[$0]) LogicalFilter(condition=[<($1, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($2, $5)], joinType=[full]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -378,11 +378,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[>($4, 100)]) +- LogicalJoin(condition=[=($3, $6)], joinType=[semi]) :- LogicalJoin(condition=[=($2, $5)], joinType=[full]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -396,12 +396,12 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(>($4, 100), IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[<($1, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($2, $5)], joinType=[full]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -410,11 +410,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[>($4, 100)]) +- LogicalJoin(condition=[=($0, $6)], joinType=[semi]) :- LogicalJoin(condition=[=($2, $5)], joinType=[full]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -428,12 +428,12 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(>($4, 100), IN($3, { LogicalProject(i=[$0]) LogicalFilter(condition=[<($1, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($2, $5)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -442,11 +442,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[>($4, 100)]) +- LogicalJoin(condition=[=($3, $6)], joinType=[semi]) :- LogicalJoin(condition=[=($2, $5)], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -460,12 +460,12 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(>($4, 100), IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[<($1, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($2, $5)], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -474,11 +474,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[>($4, 100)]) +- LogicalJoin(condition=[=($0, $6)], joinType=[semi]) :- LogicalJoin(condition=[=($2, $5)], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -492,11 +492,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), IN($3, { LogicalProject(i=[$0]) LogicalFilter(condition=[AND(>($cor0.e, $1), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -504,12 +504,12 @@ LogicalProject(i=[$0]) LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalJoin(condition=[AND(=($0, $3), >($1, $4))], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0], j=[$1]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -523,11 +523,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[AND(>($cor0.b, $1), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -536,11 +536,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[AND(=($0, $3), >($1, $4))], joinType=[semi]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(i=[$0], j=[$1]) : +- LogicalFilter(condition=[<($1, 50)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -554,11 +554,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[AND(>($cor0.e, $1), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -567,11 +567,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[AND(=($0, $6), >($4, $7))], joinType=[semi]) :- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0], j=[$1]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -585,11 +585,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), IN($3, { LogicalProject(i=[$0]) LogicalFilter(condition=[AND(>($cor0.b, $1), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -598,11 +598,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[AND(=($3, $6), >($1, $7))], joinType=[semi]) :- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0], j=[$1]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -616,12 +616,12 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(>($4, 100), IN($3, { LogicalProject(i=[$0]) LogicalFilter(condition=[<($1, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($2, $5)], joinType=[right]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -630,11 +630,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[>($4, 100)]) +- LogicalJoin(condition=[=($3, $6)], joinType=[semi]) :- LogicalJoin(condition=[=($2, $5)], joinType=[right]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -648,12 +648,12 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(>($4, 100), IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[<($1, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($2, $5)], joinType=[right]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -662,11 +662,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[>($4, 100)]) +- LogicalJoin(condition=[=($0, $6)], joinType=[semi]) :- LogicalJoin(condition=[=($2, $5)], joinType=[right]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -679,11 +679,11 @@ LogicalProject(a=[$0], f=[$5]) LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), NOT(EXISTS({ LogicalFilter(condition=[<($1, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -692,13 +692,13 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[$6], joinType=[anti]) :- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject($f0=[IS NOT NULL($0)]) +- LogicalAggregate(group=[{}], m=[MIN($0)]) +- LogicalProject(i=[true]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -711,11 +711,11 @@ LogicalProject(a=[$0], f=[$5]) LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), NOT(EXISTS({ LogicalFilter(condition=[AND(=($0, $cor0.a), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -724,11 +724,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[=($3, $0)], joinType=[anti]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(i=[$0]) : +- LogicalFilter(condition=[<($1, 50)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -741,11 +741,11 @@ LogicalProject(a=[$0], f=[$5]) LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), NOT(EXISTS({ LogicalFilter(condition=[AND(=($0, $cor0.d), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -753,12 +753,12 @@ LogicalFilter(condition=[AND(=($0, $cor0.d), <($1, 50))]) LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalJoin(condition=[=($3, $0)], joinType=[anti]) - :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -771,11 +771,11 @@ LogicalProject(a=[$0], f=[$5]) LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), NOT(EXISTS({ LogicalFilter(condition=[AND(>($0, $cor0.a), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -784,11 +784,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[>($3, $0)], joinType=[anti]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(i=[$0]) : +- LogicalFilter(condition=[<($1, 50)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -801,11 +801,11 @@ LogicalProject(a=[$0], f=[$5]) LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), NOT(EXISTS({ LogicalFilter(condition=[AND(<($0, $cor0.d), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -813,12 +813,12 @@ LogicalFilter(condition=[AND(<($0, $cor0.d), <($1, 50))]) LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalJoin(condition=[<($3, $0)], joinType=[anti]) - :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -832,11 +832,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), NOT(IN($3, { LogicalProject(i=[$0]) LogicalFilter(condition=[<($1, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -844,12 +844,12 @@ LogicalProject(i=[$0]) LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalJoin(condition=[OR(IS NULL($0), IS NULL($3), =($0, $3))], joinType=[anti]) - :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -863,11 +863,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), NOT(IN($3, { LogicalProject(i=[$0]) LogicalFilter(condition=[AND(=($cor0.e, $1), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -875,12 +875,12 @@ LogicalProject(i=[$0]) LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalJoin(condition=[AND(OR(IS NULL($0), IS NULL($3), =($0, $3)), =($1, $4))], joinType=[anti]) - :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0], j=[$1]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -894,11 +894,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), NOT(IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[<($1, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -907,11 +907,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[OR(IS NULL($0), IS NULL($3), =($0, $3))], joinType=[anti]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(i=[$0]) : +- LogicalFilter(condition=[<($1, 50)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -925,11 +925,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), NOT(IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -938,11 +938,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[AND(OR(IS NULL($0), IS NULL($3), =($0, $3)), =($1, $4))], joinType=[anti]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(i=[$0], j=[$1]) : +- LogicalFilter(condition=[<($1, 50)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -956,11 +956,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), NOT(IN($0, $4, { LogicalProject(i=[$0], j=[$1]) LogicalFilter(condition=[<($1, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -969,11 +969,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[AND(OR(IS NULL($0), IS NULL($6), =($0, $6)), OR(IS NULL($4), IS NULL($7), =($4, $7)))], joinType=[anti]) :- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0], j=[$1]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -987,11 +987,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), NOT(IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[AND(=($cor0.e, $1), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1000,11 +1000,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[AND(OR(IS NULL($0), IS NULL($6), =($0, $6)), =($4, $7))], joinType=[anti]) :- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0], j=[$1]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -1018,11 +1018,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), NOT(IN($3, { LogicalProject(i=[$0]) LogicalFilter(condition=[AND(>($cor0.e, $1), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1030,12 +1030,12 @@ LogicalProject(i=[$0]) LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalJoin(condition=[AND(OR(IS NULL($0), IS NULL($3), =($0, $3)), >($1, $4))], joinType=[anti]) - :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0], j=[$1]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -1049,11 +1049,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), NOT(IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[AND(>($cor0.b, $1), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1062,11 +1062,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[AND(OR(IS NULL($0), IS NULL($3), =($0, $3)), >($1, $4))], joinType=[anti]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(i=[$0], j=[$1]) : +- LogicalFilter(condition=[<($1, 50)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1080,11 +1080,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), NOT(IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[AND(>($cor0.e, $1), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1093,11 +1093,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[AND(OR(IS NULL($0), IS NULL($6), =($0, $6)), >($4, $7))], joinType=[anti]) :- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0], j=[$1]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -1111,11 +1111,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100), NOT(IN($3, { LogicalProject(i=[$0]) LogicalFilter(condition=[AND(>($cor0.b, $1), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1124,11 +1124,11 @@ LogicalProject(a=[$0], f=[$5]) +- LogicalFilter(condition=[AND(=($2, $5), >($4, 100))]) +- LogicalJoin(condition=[AND(OR(IS NULL($3), IS NULL($6), =($3, $6)), >($1, $7))], joinType=[anti]) :- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0], j=[$1]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinProjectTransposeRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinProjectTransposeRuleTest.xml index a3276a40450ab..87993a411a76c 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinProjectTransposeRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinProjectTransposeRuleTest.xml @@ -30,13 +30,13 @@ LogicalProject(a=[$0], f=[$4]) +- LogicalFilter(condition=[AND(>($3, 100), IN($2, { LogicalProject(i=[$0]) LogicalFilter(condition=[<($1, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))]) +- LogicalProject(a=[*($0, 2)], b=[$1], d=[+($3, 1)], e=[$4], f=[$5]) +- LogicalFilter(condition=[=($2, $5)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -47,11 +47,11 @@ LogicalProject(a=[$0], f=[$4]) :- LogicalProject(a=[*($0, 2)], b=[$1], d=[+($3, 1)], e=[$4], f=[$5]) : +- LogicalFilter(condition=[=($2, $5)]) : +- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -69,13 +69,13 @@ LogicalProject(a=[$0], f=[$4]) +- LogicalFilter(condition=[AND(>($3, 100), IN($2, { LogicalProject(i=[$0]) LogicalFilter(condition=[<($1, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))]) +- LogicalProject(a=[$0], b=[$1], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[=($2, $5)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -85,12 +85,12 @@ LogicalProject(a=[$0], f=[$4]) +- LogicalProject(a=[$0], b=[$1], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[=($2, $5)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalJoin(condition=[=($0, $3)], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinConditionEqualityTransferRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinConditionEqualityTransferRuleTest.xml index 5a9ace9c2c7e3..4acfef9acbe2c 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinConditionEqualityTransferRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinConditionEqualityTransferRuleTest.xml @@ -24,16 +24,16 @@ limitations under the License. @@ -45,16 +45,16 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) @@ -66,16 +66,16 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) @@ -89,8 +89,8 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$4], e=[$5], f=[$6]) +- LogicalJoin(condition=[AND(=($0, $4), =($0, $5), =($3, $4))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[+($1, 1)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -99,8 +99,8 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$4], e=[$5], f=[$6]) +- LogicalJoin(condition=[AND(=($0, $3), =($5, $4), =($0, $5))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[+($1, 1)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -112,16 +112,16 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) ($0, $4))], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> ($0, $4))], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -133,16 +133,16 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) ($1, $3))], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> ($1, $3), =($3, $4), =($0, $3))], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -157,19 +157,19 @@ SELECT * FROM MyTable1 WHERE EXISTS (SELECT * FROM MyTable2 WHERE a = d AND a = LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), =($cor0.a, $1))]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -185,19 +185,19 @@ SELECT * FROM MyTable1 WHERE EXISTS LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), =($cor0.a, $1), =($cor0.b, $0))]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -211,19 +211,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.a, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -237,19 +237,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.a, $1), =($cor0.b, $0))]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -264,19 +264,19 @@ SELECT * FROM MyTable1 WHERE EXISTS (SELECT * FROM MyTable2 WHERE a = d AND a > LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), >($cor0.a, $1))]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> ($0, $4))], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(d=[$0], e=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -292,19 +292,19 @@ SELECT * FROM MyTable1 WHERE EXISTS LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), =($cor0.a, $1), >($cor0.b, $0))]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> ($1, $3), =($3, $4), =($0, $3))], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(d=[$0], e=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -318,19 +318,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($cor0.a, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> ($0, $4))], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(d=[$0], e=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -344,19 +344,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[AND(>($cor0.a, $1), =($cor0.b, $0))]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> ($0, $4), =($0, $1), =($0, $3))], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(d=[$0], e=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinConditionTypeCoerceRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinConditionTypeCoerceRuleTest.xml index 1802dc6fb50d1..6af67a2837a65 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinConditionTypeCoerceRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinConditionTypeCoerceRuleTest.xml @@ -25,19 +25,19 @@ limitations under the License. LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalFilter(condition=[IN($3, { LogicalProject(e=[$4]) - LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a, b, c, d, e)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T2]]) })]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] ]]> @@ -50,9 +50,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTE LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalFilter(condition=[IN($2, { LogicalProject(e=[$4]) - LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a, b, c, d, e)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T2]]) })]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] ]]> @@ -61,10 +61,10 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTE +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalJoin(condition=[=($5, $7)], joinType=[semi]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e, DOUBLE c0)] :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], c0=[CAST($2):DOUBLE]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e, DOUBLE c0)] - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalProject(e=[$0], e0=[CAST($0):DOUBLE]), rowType=[RecordType(DECIMAL(38, 18) e, DOUBLE e0)] +- LogicalProject(e=[$4]), rowType=[RecordType(DECIMAL(38, 18) e)] - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] ]]> @@ -77,9 +77,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTE LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalFilter(condition=[IN($0, { LogicalProject(e=[$4]) - LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a, b, c, d, e)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T2]]) })]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] ]]> @@ -88,9 +88,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTE +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalJoin(condition=[=($5, $6)], joinType=[semi]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e, DECIMAL(38, 18) a0)] :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], a0=[CAST($0):DECIMAL(38, 18)]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e, DECIMAL(38, 18) a0)] - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalProject(e=[$4]), rowType=[RecordType(DECIMAL(38, 18) e)] - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] ]]> @@ -103,19 +103,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTE LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalFilter(condition=[IN($2, { LogicalProject(d=[$3]) - LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a, b, c, d, e)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T2]]) })]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] ]]> @@ -128,9 +128,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTE LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$3]) - LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a, b, c, d, e)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T2]]) })]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] ]]> @@ -139,9 +139,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTE +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalJoin(condition=[=($5, $6)], joinType=[semi]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e, DOUBLE a0)] :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], a0=[CAST($0):DOUBLE]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e, DOUBLE a0)] - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalProject(d=[$3]), rowType=[RecordType(DOUBLE d)] - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] ]]> @@ -154,9 +154,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTE LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalFilter(condition=[IN($0, { LogicalProject(c=[$2]) - LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a, b, c, d, e)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T2]]) })]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] ]]> @@ -165,9 +165,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTE +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalJoin(condition=[=($5, $6)], joinType=[semi]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e, FLOAT a0)] :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], a0=[CAST($0):FLOAT]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e, FLOAT a0)] - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalProject(c=[$2]), rowType=[RecordType(FLOAT c)] - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] ]]> @@ -180,9 +180,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTE LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalFilter(condition=[IN($0, { LogicalProject(b=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a, b, c, d, e)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T2]]) })]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] ]]> @@ -191,9 +191,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTE +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalJoin(condition=[=($5, $6)], joinType=[semi]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e, BIGINT a0)] :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], a0=[CAST($0):BIGINT]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e, BIGINT a0)] - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalProject(b=[$1]), rowType=[RecordType(BIGINT b)] - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] ]]> @@ -207,10 +207,10 @@ LogicalProject(a=[$0]), rowType=[RecordType(INTEGER a)] +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], cnt=[$6]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e, BIGINT cnt)] +- LogicalJoin(condition=[=($5, $6)], joinType=[left]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e, BIGINT a0, BIGINT cnt)] :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], a0=[CAST($0):BIGINT]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e, BIGINT a0)] - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalAggregate(group=[{}], cnt=[COUNT()]), rowType=[RecordType(BIGINT cnt)] +- LogicalProject($f0=[0]), rowType=[RecordType(INTEGER $f0)] - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] ]]> @@ -219,10 +219,10 @@ LogicalProject(a=[$0]), rowType=[RecordType(INTEGER a)] +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], cnt=[$6]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e, BIGINT cnt)] +- LogicalJoin(condition=[=($5, $6)], joinType=[left]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e, BIGINT a0, BIGINT cnt)] :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], a0=[CAST($0):BIGINT]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e, BIGINT a0)] - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalAggregate(group=[{}], cnt=[COUNT()]), rowType=[RecordType(BIGINT cnt)] +- LogicalProject($f0=[0]), rowType=[RecordType(INTEGER $f0)] - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a, b, c, d, e)]]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] ]]> @@ -245,23 +245,23 @@ LogicalProject(t3a=[$0], t3b=[$1]), rowType=[RecordType(VARCHAR(2147483647) t3a, +- LogicalFilter(condition=[IN($2, { LogicalProject(t4b=[$1]) LogicalFilter(condition=[OR(=($cor0.t3a, $0), >($cor0.t3b, $1))]) - LogicalTableScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(t4a, t4b, t4c)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T4]]) })], variablesSet=[[$cor0]]), rowType=[RecordType(VARCHAR(2147483647) t3a, SMALLINT t3b, INTEGER t3c)] - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(t3a, t3b, t3c)]]]), rowType=[RecordType(VARCHAR(2147483647) t3a, SMALLINT t3b, INTEGER t3c)] + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]), rowType=[RecordType(VARCHAR(2147483647) t3a, SMALLINT t3b, INTEGER t3c)] ]]> ($4, $1))], joinType=[inner]), rowType=[RecordType(VARCHAR(2147483647) t4a, SMALLINT t4b, INTEGER t4c, VARCHAR(2147483647) t3a, SMALLINT t3b)] - :- LogicalTableScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(t4a, t4b, t4c)]]]), rowType=[RecordType(VARCHAR(2147483647) t4a, SMALLINT t4b, INTEGER t4c)] + :- LogicalTableScan(table=[[default_catalog, default_database, T4]]), rowType=[RecordType(VARCHAR(2147483647) t4a, SMALLINT t4b, INTEGER t4c)] +- LogicalAggregate(group=[{0, 1}]), rowType=[RecordType(VARCHAR(2147483647) t3a, SMALLINT t3b)] +- LogicalProject(t3a=[$0], t3b=[$1]), rowType=[RecordType(VARCHAR(2147483647) t3a, SMALLINT t3b)] - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(t3a, t3b, t3c)]]]), rowType=[RecordType(VARCHAR(2147483647) t3a, SMALLINT t3b, INTEGER t3c)] + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]), rowType=[RecordType(VARCHAR(2147483647) t3a, SMALLINT t3b, INTEGER t3c)] ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinDependentConditionDerivationRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinDependentConditionDerivationRuleTest.xml index be18f2318c9fb..723e6fd0b971d 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinDependentConditionDerivationRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinDependentConditionDerivationRuleTest.xml @@ -25,16 +25,16 @@ limitations under the License. LogicalProject(a=[$0], d=[$3]) +- LogicalFilter(condition=[OR(AND(=($1, $4), =($0, 0)), AND(=($0, 1), =($3, 2)), AND(=($0, 2), =($3, 1)))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -47,16 +47,16 @@ LogicalProject(a=[$0], d=[$3]) LogicalProject(a=[$0], d=[$3]) +- LogicalFilter(condition=[AND(=($1, $4), OR(AND(=($0, 1), =($3, 2)), AND(=($0, 2), =($3, 1))))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -69,16 +69,16 @@ LogicalProject(a=[$0], d=[$3]) LogicalProject(a=[$0], d=[$3]) +- LogicalFilter(condition=[OR(=($1, $4), AND(=($0, 1), =($3, 2)), AND(=($0, 2), =($3, 1)))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -91,16 +91,16 @@ LogicalProject(a=[$0], d=[$3]) LogicalProject(a=[$0], d=[$3]) +- LogicalFilter(condition=[OR(AND(=($0, 1), =($1, 1), =($3, 2), =($4, 2)), AND(=($0, 2), =($1, 2), =($3, 1), =($4, 1)))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -113,16 +113,16 @@ LogicalProject(a=[$0], d=[$3]) LogicalProject(a=[$0], d=[$3]) +- LogicalFilter(condition=[OR(AND(=($0, 1), =($3, 2)), AND(=($0, 2), =($3, 1)))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -141,23 +141,23 @@ OR LogicalProject(a=[$0], d=[$6]) +- LogicalFilter(condition=[OR(AND(=($0, 1), =($1, 1), =($3, 2), =($7, 2)), AND(=($0, 2), =($4, 2), =($6, 1), =($7, 1)))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], g=[$6], h=[$7]) +- LogicalFilter(condition=[=($0, $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -170,16 +170,16 @@ LogicalProject(a=[$0], d=[$6]) LogicalProject(a=[$0], d=[$3]) +- LogicalFilter(condition=[OR(AND(=($0, 1), =($1, 1), =($3, 2), =($4, 2)), AND(=($3, 1), =($4, 1)))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -192,16 +192,16 @@ LogicalProject(a=[$0], d=[$3]) LogicalProject(a=[$0], d=[$3]) +- LogicalFilter(condition=[OR(AND(=($0, 1), =($3, 2)), AND(=($0, 2), =($3, 1)))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinPushExpressionsRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinPushExpressionsRuleTest.xml index 4532fc2fef79f..1080952171fdd 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinPushExpressionsRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinPushExpressionsRuleTest.xml @@ -25,9 +25,9 @@ limitations under the License. LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=(CAST($cor0.b):INTEGER, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -36,10 +36,10 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[=($3, $4)], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], c=[$2], b0=[CAST($1):INTEGER]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(d=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -53,9 +53,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=(+($cor0.b, 1), $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -64,10 +64,10 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(=($0, $4), =($3, $5))], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[+($1, 1)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(d=[$0], e=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -81,9 +81,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=(+($cor0.b, 1), $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -92,10 +92,10 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(OR(IS NULL($0), IS NULL($4), =($0, $4)), =($3, $5))], joinType=[anti]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[+($1, 1)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(d=[$0], e=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -108,9 +108,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[=(CAST($cor0.b):INTEGER, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -119,10 +119,10 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[=($3, $4)], joinType=[anti]) :- LogicalProject(a=[$0], b=[$1], c=[$2], b0=[CAST($1):INTEGER]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(d=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- 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/logical/LogicalCorrelateToJoinFromTemporalTableRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalCorrelateToJoinFromTemporalTableRuleTest.xml index d365bf4feaf6a..f55e06f7e1773 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalCorrelateToJoinFromTemporalTableRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalCorrelateToJoinFromTemporalTableRuleTest.xml @@ -83,7 +83,8 @@ LogicalProject(id=[$0], mount=[$1], proctime=[$2], rowtime=[$3], id0=[$4], rate= : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [CollectionTableSource(id, mount, rowtime)]]]) +- LogicalFilter(condition=[AND(=($cor0.id, $0), >($1, 10))]) +- LogicalSnapshot(period=[$cor0.proctime]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(id, rate, rowtime)]]]) + +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[$2]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -94,7 +95,8 @@ LogicalProject(id=[$0], mount=[$1], proctime=[$2], rowtime=[$3], id0=[$4], rate= : +- LogicalProject(id=[$0], mount=[$1], proctime=[PROCTIME()], rowtime=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [CollectionTableSource(id, mount, rowtime)]]]) +- LogicalSnapshot(period=[$cor0.proctime]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(id, rate, rowtime)]]]) + +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[$2]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -139,7 +141,8 @@ LogicalProject(id=[$0], mount=[$1], proctime=[$2], rowtime=[$3], id0=[$4], rate= : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [CollectionTableSource(id, mount, rowtime)]]]) +- LogicalFilter(condition=[AND(=($cor0.id, $0), >($1, 10))]) +- LogicalSnapshot(period=[$cor0.rowtime]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(id, rate, rowtime)]]]) + +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[$2]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -150,7 +153,8 @@ LogicalProject(id=[$0], mount=[$1], proctime=[$2], rowtime=[$3], id0=[$4], rate= : +- LogicalProject(id=[$0], mount=[$1], proctime=[PROCTIME()], rowtime=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [CollectionTableSource(id, mount, rowtime)]]]) +- LogicalSnapshot(period=[$cor0.rowtime]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(id, rate, rowtime)]]]) + +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[$2]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -167,7 +171,8 @@ LogicalProject(id=[$0], mount=[$1], proctime=[$2], rowtime=[$3], id0=[$4], rate= : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [CollectionTableSource(id, mount, rowtime)]]]) +- LogicalFilter(condition=[AND(=($cor0.id, $0), >($1, 10))]) +- LogicalSnapshot(period=[$cor0.rowtime]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(id, rate, rowtime)]]]) + +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[$2]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> @@ -178,7 +183,8 @@ LogicalProject(id=[$0], mount=[$1], proctime=[$2], rowtime=[$3], id0=[$4], rate= : +- LogicalProject(id=[$0], mount=[$1], proctime=[PROCTIME()], rowtime=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [CollectionTableSource(id, mount, rowtime)]]]) +- LogicalSnapshot(period=[$cor0.rowtime]) - +- LogicalTableScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(id, rate, rowtime)]]]) + +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[$2]) + +- LogicalTableScan(table=[[default_catalog, default_database, T3]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml index 31765f140a4b0..edd58ca7be47a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml @@ -24,7 +24,7 @@ limitations under the License. @@ -109,7 +109,7 @@ LogicalProject(EXPR$0=[$0]) @@ -142,7 +142,7 @@ LogicalProject(a=[$0], b=[$1], x=[$2], y=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalProject(a=[$0], b=[$1]) : +- LogicalFilter(condition=[<($0, 3)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(x=[$0], y=[$1]) +- Uncollect +- LogicalProject(b=[$cor0.b]) @@ -156,7 +156,7 @@ LogicalProject(a=[$0], b=[$1], x=[$2], y=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalProject(a=[$0], b=[$1]) : +- LogicalFilter(condition=[<($0, 3)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(x=[$0], y=[$1]) +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2)]) ]]> @@ -204,7 +204,7 @@ LogicalProject(a=[$0], s=[$2]) @@ -263,7 +263,7 @@ LogicalProject(b=[$0], id=[$2], point=[$3]) @@ -289,7 +289,7 @@ LogicalProject(a=[$0], b=[$1], s=[$3]) LogicalProject(a=[$0], b=[$1], s=[$2], t=[$3]) +- LogicalFilter(condition=[>($2, 13)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(s=[$0], t=[$1]) +- Uncollect +- LogicalProject(b=[$cor0.b]) @@ -301,7 +301,7 @@ LogicalProject(a=[$0], b=[$1], s=[$2], t=[$3]) LogicalProject(a=[$0], b=[$1], s=[$2], t=[$3]) +- LogicalFilter(condition=[>($2, 13)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(s=[$0], t=[$1]) +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2)]) ]]> @@ -316,7 +316,7 @@ LogicalProject(a=[$0], b=[$1], s=[$2], t=[$3]) LogicalProject(a=[$0], b=[$1], _1=[$2], _2=[$3]) +- LogicalFilter(condition=[>($2, 1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- Uncollect +- LogicalProject(b=[$cor0.b]) +- LogicalValues(tuples=[[{ 0 }]]) @@ -327,7 +327,7 @@ LogicalProject(a=[$0], b=[$1], _1=[$2], _2=[$3]) LogicalProject(a=[$0], b=[$1], _1=[$2], _2=[$3]) +- LogicalFilter(condition=[>($2, 1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2)]) ]]> @@ -353,7 +353,7 @@ LogicalProject(a=[$0], b1=[$1], b2=[$2]) +- LogicalFilter(condition=[>=($2, 12)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(b1=[$0], b2=[$1]) +- Uncollect +- LogicalProject(b=[$cor0.b]) @@ -368,7 +368,7 @@ LogicalProject(a=[$0], b1=[$1], b2=[$2]) +- LogicalFilter(condition=[>=($2, 12)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(b1=[$0], b2=[$1]) +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2)]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ProjectPruneAggregateCallRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ProjectPruneAggregateCallRuleTest.xml index 1d65ace38666a..c1a88eef41e5b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ProjectPruneAggregateCallRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ProjectPruneAggregateCallRuleTest.xml @@ -25,7 +25,7 @@ limitations under the License. LogicalProject(EXPR$0=[1]) +- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)], EXPR$1=[COUNT()]) +- LogicalProject(a1=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) ]]> @@ -33,7 +33,7 @@ LogicalProject(EXPR$0=[1]) LogicalProject(EXPR$0=[1]) +- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)]) +- LogicalProject(a1=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) ]]> @@ -47,14 +47,14 @@ LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3]) +- LogicalFilter(condition=[EXISTS({ LogicalAggregate(group=[{}], EXPR$0=[SUM($0)], EXPR$1=[COUNT()]) LogicalProject(a1=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T1]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2, d2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -69,14 +69,14 @@ LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3]) LogicalAggregate(group=[{}], EXPR$0=[SUM($0)], EXPR$1=[COUNT()]) LogicalProject(a1=[$0]) LogicalFilter(condition=[=(1, 2)]) - LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T1]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2, d2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -89,7 +89,7 @@ LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTab LogicalProject(EXPR$0=[1]) +- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)]) +- LogicalProject(a1=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) ]]> @@ -97,7 +97,7 @@ LogicalProject(EXPR$0=[1]) LogicalProject(EXPR$0=[1]) +- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)]) +- LogicalProject(a1=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) ]]> @@ -111,14 +111,14 @@ LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3]) +- LogicalFilter(condition=[EXISTS({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalProject($f0=[0]) - LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T1]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2, d2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -133,14 +133,14 @@ LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3]) LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalProject($f0=[0]) LogicalFilter(condition=[=(1, 2)]) - LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + LogicalTableScan(table=[[default_catalog, default_database, T1]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2, d2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -157,14 +157,14 @@ WHERE c1 > 10 LogicalProject(a1=[$0], c1=[$2]) +- LogicalFilter(condition=[>($2, 10)]) +- LogicalAggregate(group=[{0, 1}], c1=[COUNT($2)], d1=[SUM($3)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) ]]> ($1, 10)]) +- LogicalAggregate(group=[{0}], c1=[COUNT($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) ]]> @@ -181,7 +181,7 @@ WHERE c1 > 10 LogicalProject(c1=[$2], a1=[$0]) +- LogicalFilter(condition=[>($2, 10)]) +- LogicalAggregate(group=[{0, 1}], c1=[COUNT($2)], d1=[SUM($3)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) ]]> @@ -189,7 +189,7 @@ LogicalProject(c1=[$2], a1=[$0]) LogicalProject(c1=[$1], a1=[$0]) +- LogicalFilter(condition=[>($1, 10)]) +- LogicalAggregate(group=[{0}], c1=[COUNT($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) ]]> @@ -204,13 +204,13 @@ SELECT a1, c1 FROM @@ -225,14 +225,14 @@ SELECT c1, a1 FROM @@ -249,14 +249,14 @@ WHERE d2 > 0 LogicalProject(a2=[$0], b2=[$1], d2=[$3]) +- LogicalFilter(condition=[>($3, 0)]) +- LogicalAggregate(group=[{0, 1}], c2=[COUNT($2)], d2=[SUM($3)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2, d2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> ($2, 0)]) +- LogicalAggregate(group=[{0, 1}], d2=[SUM($3)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2, d2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -273,7 +273,7 @@ WHERE d2 > 0 LogicalProject(b2=[$1], a2=[$0], d2=[$3]) +- LogicalFilter(condition=[>($3, 0)]) +- LogicalAggregate(group=[{0, 1}], c2=[COUNT($2)], d2=[SUM($3)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2, d2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -281,7 +281,7 @@ LogicalProject(b2=[$1], a2=[$0], d2=[$3]) LogicalProject(b2=[$1], a2=[$0], d2=[$2]) +- LogicalFilter(condition=[>($2, 0)]) +- LogicalAggregate(group=[{0, 1}], d2=[SUM($3)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2, d2)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -296,13 +296,13 @@ SELECT a2, b2, d2 FROM @@ -317,14 +317,14 @@ SELECT b2, a2, d2 FROM @@ -339,14 +339,14 @@ SELECT a2 as a, b2, d2 FROM diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ProjectSemiAntiJoinTransposeRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ProjectSemiAntiJoinTransposeRuleTest.xml index a4f8e0b18a080..0d99c0ed451cb 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ProjectSemiAntiJoinTransposeRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ProjectSemiAntiJoinTransposeRuleTest.xml @@ -25,18 +25,18 @@ limitations under the License. LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -49,18 +49,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(EXPR$0=[+($1, 1)], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -74,19 +74,19 @@ LogicalProject(c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -99,18 +99,18 @@ LogicalProject(c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -123,18 +123,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(EXPR$0=[+($1, 1)], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -148,19 +148,19 @@ LogicalProject(c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -173,18 +173,18 @@ LogicalProject(c=[$2]) LogicalProject(a=[$0]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -197,9 +197,9 @@ LogicalJoin(condition=[OR(IS NULL($0), IS NULL($1), =($0, $1))], joinType=[anti] LogicalProject(EXPR$0=[+($0, 1)]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -207,9 +207,9 @@ LogicalProject(d=[$0]) LogicalProject(EXPR$0=[+($0, 1)]) +- LogicalJoin(condition=[OR(IS NULL($0), IS NULL($1), =($0, $1))], joinType=[anti]) :- LogicalProject(a=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -222,18 +222,18 @@ LogicalProject(EXPR$0=[+($0, 1)]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -246,9 +246,9 @@ LogicalJoin(condition=[OR(IS NULL($0), IS NULL($2), =($0, $2))], joinType=[anti] LogicalProject(b=[$1], a=[$0]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -256,9 +256,9 @@ LogicalProject(d=[$0]) LogicalProject(b=[$1], a=[$0]) +- LogicalJoin(condition=[OR(IS NULL($0), IS NULL($2), =($0, $2))], joinType=[anti]) :- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -271,12 +271,12 @@ LogicalProject(b=[$1], a=[$0]) LogicalProject(d=[$3]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) }))]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -286,10 +286,10 @@ LogicalProject(d=[$1]) :- LogicalProject(a=[$0], d=[$3]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) : +- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) +- LogicalProject(d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -302,32 +302,32 @@ LogicalProject(d=[$1]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(i, j, k)]]]) +LogicalTableScan(table=[[default_catalog, default_database, MyTable3]]) })]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -340,18 +340,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -364,9 +364,9 @@ LogicalJoin(condition=[=($0, $1)], joinType=[semi]) LogicalProject(EXPR$0=[+($0, 1)]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -374,9 +374,9 @@ LogicalProject(d=[$0]) LogicalProject(EXPR$0=[+($0, 1)]) +- LogicalJoin(condition=[=($0, $1)], joinType=[semi]) :- LogicalProject(a=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -389,18 +389,18 @@ LogicalProject(EXPR$0=[+($0, 1)]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -413,9 +413,9 @@ LogicalJoin(condition=[=($0, $2)], joinType=[semi]) LogicalProject(b=[$1], a=[$0]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -423,9 +423,9 @@ LogicalProject(d=[$0]) LogicalProject(b=[$1], a=[$0]) +- LogicalJoin(condition=[=($0, $2)], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -438,12 +438,12 @@ LogicalProject(b=[$1], a=[$0]) LogicalProject(d=[$3]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) })]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -453,10 +453,10 @@ LogicalProject(d=[$1]) :- LogicalProject(a=[$0], d=[$3]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) : +- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) +- LogicalProject(d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.xml index f1007afea5f52..086f00174ceed 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.xml @@ -23,7 +23,7 @@ limitations under the License. @@ -31,7 +31,7 @@ LogicalProject(EXPR$0=[pandasFunc3(pandasFunc2(+($0, pandasFunc1($0, $2)), $1), FlinkLogicalCalc(select=[pandasFunc3(pandasFunc2(f0, b), c) AS EXPR$0]) +- FlinkLogicalCalc(select=[b, c, +(a, f0) AS f0]) +- FlinkLogicalCalc(select=[b, c, a, pandasFunc1(a, c) AS f0]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -42,7 +42,7 @@ FlinkLogicalCalc(select=[pandasFunc3(pandasFunc2(f0, b), c) AS EXPR$0]) @@ -50,7 +50,7 @@ LogicalProject(EXPR$0=[pyFunc3(pyFunc2(+($0, pyFunc1($0, $2)), $1), $2)]) FlinkLogicalCalc(select=[pyFunc3(pyFunc2(f0, b), c) AS EXPR$0]) +- FlinkLogicalCalc(select=[b, c, +(a, f0) AS f0]) +- FlinkLogicalCalc(select=[b, c, a, pyFunc1(a, c) AS f0]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -61,14 +61,14 @@ FlinkLogicalCalc(select=[pyFunc3(pyFunc2(f0, b), c) AS EXPR$0]) @@ -79,14 +79,14 @@ FlinkLogicalCalc(select=[a, pyFunc1(b, pyFunc1(c, f0)) AS EXPR$1]) @@ -97,14 +97,14 @@ FlinkLogicalCalc(select=[f00 AS EXPR$0, +(f0, 1) AS EXPR$1]) @@ -115,13 +115,13 @@ FlinkLogicalCalc(select=[a, b, f0 AS EXPR$2, 1 AS EXPR$3]) @@ -133,14 +133,14 @@ FlinkLogicalCalc(select=[pandasFunc1(a, b) AS EXPR$0]) @@ -151,13 +151,13 @@ FlinkLogicalCalc(select=[a, b], where=[f0]) @@ -169,14 +169,14 @@ FlinkLogicalCalc(select=[pyFunc1(a, b) AS EXPR$0]) @@ -187,14 +187,14 @@ FlinkLogicalCalc(select=[a, b], where=[f0]) @@ -205,14 +205,14 @@ FlinkLogicalCalc(select=[+(f0, 1) AS EXPR$0]) @@ -224,7 +224,7 @@ FlinkLogicalCalc(select=[+(f0, 1) AS EXPR$0]) @@ -232,7 +232,7 @@ LogicalProject(EXPR$0=[pandasFunc1($0, $1)]) FlinkLogicalCalc(select=[pandasFunc1(a, b) AS EXPR$0]) +- FlinkLogicalCalc(select=[a, b], where=[f0]) +- FlinkLogicalCalc(select=[a, b, pandasFunc4(a, c) AS f0]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -243,7 +243,7 @@ FlinkLogicalCalc(select=[pandasFunc1(a, b) AS EXPR$0]) @@ -251,7 +251,7 @@ LogicalProject(EXPR$0=[pandasFunc1($0, $1)], EXPR$1=[+(pyFunc1($0, $2), 1)], EXP FlinkLogicalCalc(select=[f0 AS EXPR$0, +(f1, 1) AS EXPR$1, +(a, 1) AS EXPR$2]) +- FlinkLogicalCalc(select=[a, f0, pyFunc1(a, c) AS f1]) +- FlinkLogicalCalc(select=[a, c, pandasFunc1(a, b) AS f0]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -262,14 +262,14 @@ FlinkLogicalCalc(select=[f0 AS EXPR$0, +(f1, 1) AS EXPR$1, +(a, 1) AS EXPR$2]) @@ -281,7 +281,7 @@ FlinkLogicalCalc(select=[f0 AS EXPR$0, +(c, 1) AS EXPR$1]) (pandasFunc2($0, $2), 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -290,7 +290,7 @@ FlinkLogicalCalc(select=[f0 AS EXPR$0, +(c, 1) AS EXPR$1]) +- FlinkLogicalCalc(select=[c, pandasFunc1(a, b) AS f0]) +- FlinkLogicalCalc(select=[c, a, b], where=[>(f0, 0)]) +- FlinkLogicalCalc(select=[a, b, c, pandasFunc2(a, c) AS f0]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -301,14 +301,14 @@ FlinkLogicalCalc(select=[f0 AS EXPR$0, +(c, 1) AS EXPR$1]) @@ -319,14 +319,14 @@ FlinkLogicalCalc(select=[a, pandasFunc1(b, f0) AS EXPR$1]) @@ -337,7 +337,7 @@ FlinkLogicalCalc(select=[a, pyFunc1(b, f0) AS EXPR$1]) @@ -345,7 +345,7 @@ LogicalProject(EXPR$0=[+(pyFunc1($0, pandasFunc1($0, $1)), 1)]) FlinkLogicalCalc(select=[+(f0, 1) AS EXPR$0]) +- FlinkLogicalCalc(select=[pyFunc1(a, f0) AS f0]) +- FlinkLogicalCalc(select=[a, pandasFunc1(a, b) AS f0]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -357,7 +357,7 @@ FlinkLogicalCalc(select=[+(f0, 1) AS EXPR$0]) @@ -365,7 +365,7 @@ LogicalProject(EXPR$0=[pyFunc1($0, $1)]) FlinkLogicalCalc(select=[pyFunc1(a, b) AS EXPR$0]) +- FlinkLogicalCalc(select=[a, b], where=[f0]) +- FlinkLogicalCalc(select=[a, b, pyFunc4(a, c) AS f0]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -376,14 +376,14 @@ FlinkLogicalCalc(select=[pyFunc1(a, b) AS EXPR$0]) @@ -395,7 +395,7 @@ FlinkLogicalCalc(select=[f0 AS EXPR$0, +(c, 1) AS EXPR$1]) (pyFunc2($0, $2), 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -404,7 +404,7 @@ FlinkLogicalCalc(select=[f0 AS EXPR$0, +(c, 1) AS EXPR$1]) +- FlinkLogicalCalc(select=[c, pyFunc1(a, b) AS f0]) +- FlinkLogicalCalc(select=[c, a, b], where=[>(f0, 0)]) +- FlinkLogicalCalc(select=[a, b, c, pyFunc2(a, c) AS f0]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -416,7 +416,7 @@ FlinkLogicalCalc(select=[f0 AS EXPR$0, +(c, 1) AS EXPR$1]) @@ -424,7 +424,7 @@ LogicalProject(f0=[$0.f0], f1=[$0.f1]) FlinkLogicalCalc(select=[f0.f0 AS f0, f0.f1 AS f1]) +- FlinkLogicalCalc(select=[pyFunc5(f0) AS f0]) +- FlinkLogicalCalc(select=[d._1 AS f0]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -436,14 +436,14 @@ FlinkLogicalCalc(select=[f0.f0 AS f0, f0.f1 AS f1]) (+($0, 1), 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> (+(a, 1), 0)]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -455,7 +455,7 @@ FlinkLogicalCalc(select=[a, pyFunc1(b, f0) AS EXPR$1]) ($1, 0))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -463,7 +463,7 @@ LogicalProject(EXPR$0=[+($0, 1)]) FlinkLogicalCalc(select=[+(a, 1) AS EXPR$0], where=[AND(IS NULL(f0.f0), >(b, 0))]) +- FlinkLogicalCalc(select=[a, b, RowJavaFunc(f0.f0) AS f0]) +- FlinkLogicalCalc(select=[a, b, pyFunc5(a) AS f0]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -475,14 +475,14 @@ FlinkLogicalCalc(select=[+(a, 1) AS EXPR$0], where=[AND(IS NULL(f0.f0), >(b, 0)) @@ -494,14 +494,14 @@ FlinkLogicalCalc(select=[f0.f0 AS f0, f0.f1 AS f1]) @@ -512,14 +512,14 @@ FlinkLogicalCalc(select=[f0.f0 AS f0, f0.f1 AS f1, f1.f0 AS f00, f1.f1 AS f10]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRuleTest.xml index 90db9c5005c42..a7f69ad01c728 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRuleTest.xml @@ -24,7 +24,7 @@ limitations under the License. @@ -33,7 +33,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], x=[$3]) FlinkLogicalCalc(select=[a, b, c, EXPR$0]) +- FlinkLogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) :- FlinkLogicalCalc(select=[a, b, c, pyFunc(c) AS f0]) - : +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) +- FlinkLogicalTableFunctionScan(invocation=[javaFunc($3)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) ]]> @@ -46,7 +46,7 @@ FlinkLogicalCalc(select=[a, b, c, EXPR$0]) @@ -55,7 +55,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], x=[$3], y=[$4]) FlinkLogicalCalc(select=[a, b, c, x, y]) +- FlinkLogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{0, 1, 2}]) :- FlinkLogicalCalc(select=[a, b, c, *(a, a) AS f0]) - : +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) +- FlinkLogicalTableFunctionScan(invocation=[func($3, pyFunc($1, $2))], rowType=[RecordType:peek_no_expand(INTEGER x, INTEGER y)]) ]]> @@ -68,7 +68,7 @@ FlinkLogicalCalc(select=[a, b, c, x, y]) @@ -78,7 +78,7 @@ FlinkLogicalCalc(select=[a, b, c, EXPR$0 AS x]) +- FlinkLogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{3}]) :- FlinkLogicalCalc(select=[a, b, c, d, pyFunc(f0) AS f0]) : +- FlinkLogicalCalc(select=[a, b, c, d, d._1 AS f0]) - : +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + : +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) +- FlinkLogicalTableFunctionScan(invocation=[javaFunc($4)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) ]]> @@ -91,7 +91,7 @@ FlinkLogicalCalc(select=[a, b, c, EXPR$0 AS x]) @@ -100,7 +100,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], x=[$4], y=[$5]) FlinkLogicalCalc(select=[a, b, c, x, y]) +- FlinkLogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{0, 2, 3}]) :- FlinkLogicalCalc(select=[a, b, c, d, *(d._1, a) AS f0, d._2 AS f1]) - : +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + : +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) +- FlinkLogicalTableFunctionScan(invocation=[func($4, pyFunc($5, $2))], rowType=[RecordType:peek_no_expand(INTEGER x, INTEGER y)]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonMapMergeRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonMapMergeRuleTest.xml index 54e0a85e09cbd..c0576ef081fe5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonMapMergeRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonMapMergeRuleTest.xml @@ -20,14 +20,14 @@ limitations under the License. @@ -35,7 +35,7 @@ FlinkLogicalCalc(select=[f0.f0 AS f0, f0.f1 AS f1]) @@ -44,7 +44,7 @@ FlinkLogicalCalc(select=[f0.f0 AS f0, f0.f1 AS f1]) +- FlinkLogicalCalc(select=[*org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions$RowPandasScalarFunction$fc2ab005eb7004743f2fd6dcc3fab1ca*(f0, f1) AS f0]) +- FlinkLogicalCalc(select=[f0.f0 AS f0, f0.f1 AS f1]) +- FlinkLogicalCalc(select=[*org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions$RowPythonScalarFunction$6d0a5b48d8e66954803fdd965425adec*(a, b, c) AS f0]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, source, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, source]], fields=[a, b, c]) ]]> @@ -53,7 +53,7 @@ FlinkLogicalCalc(select=[f0.f0 AS f0, f0.f1 AS f1]) @@ -61,7 +61,7 @@ LogicalFilter(condition=[*org.apache.flink.table.planner.runtime.utils.JavaUserD FlinkLogicalCalc(select=[a], where=[f0]) +- FlinkLogicalCalc(select=[a, *org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions$BooleanPythonScalarFunction$79babfe6cb7eb9b3766addebed5ba23c*(f0, a) AS f0]) +- FlinkLogicalCalc(select=[a, +(a, 1) AS f0]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, source, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, source]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RemoveSingleAggregateRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RemoveSingleAggregateRuleTest.xml index a864eaaeef12a..11e9a158b070e 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RemoveSingleAggregateRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RemoveSingleAggregateRuleTest.xml @@ -27,9 +27,9 @@ LogicalProject(EXPR$0=[-($0, 1)]) LogicalAggregate(group=[{}], agg#0=[COUNT($0)]) LogicalProject(x=[$0]) LogicalFilter(condition=[=($1, $cor0.i)]) - LogicalTableScan(table=[[default_catalog, default_database, foo, source: [TestTableSource(x, y)]]]) + LogicalTableScan(table=[[default_catalog, default_database, foo]]) })]) -+- LogicalTableScan(table=[[default_catalog, default_database, bar, source: [TestTableSource(i, s)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, bar]]) ]]> @@ -38,12 +38,12 @@ Calc(select=[-(CASE(IS NULL($f1), 0, $f1), 1) AS EXPR$0]) +- HashJoin(joinType=[LeftOuterJoin], where=[=(i, y)], select=[i, y, $f1], build=[right]) :- Exchange(distribution=[hash[i]]) : +- Calc(select=[i]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, bar, source: [TestTableSource(i, s)]]], fields=[i, s]) + : +- TableSourceScan(table=[[default_catalog, default_database, bar]], fields=[i, s]) +- HashAggregate(isMerge=[true], groupBy=[y], select=[y, Final_COUNT(count$0) AS $f1]) +- Exchange(distribution=[hash[y]]) +- LocalHashAggregate(groupBy=[y], select=[y, Partial_COUNT(x) AS count$0]) +- Calc(select=[x, y], where=[IS NOT NULL(y)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, foo, source: [TestTableSource(x, y)]]], fields=[x, y]) + +- TableSourceScan(table=[[default_catalog, default_database, foo]], fields=[x, y]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ReplaceIntersectWithSemiJoinRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ReplaceIntersectWithSemiJoinRuleTest.xml index 50cdf35809f05..d6e3d11936d51 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ReplaceIntersectWithSemiJoinRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ReplaceIntersectWithSemiJoinRuleTest.xml @@ -24,9 +24,9 @@ limitations under the License. @@ -34,9 +34,9 @@ LogicalIntersect(all=[false]) LogicalAggregate(group=[{0}]) +- LogicalJoin(condition=[IS NOT DISTINCT FROM($0, $1)], joinType=[semi]) :- LogicalProject(c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -49,9 +49,9 @@ LogicalAggregate(group=[{0}]) LogicalIntersect(all=[false]) :- LogicalProject(c=[$2]) : +- LogicalFilter(condition=[=(1, 0)]) -: +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -60,9 +60,9 @@ LogicalAggregate(group=[{0}]) +- LogicalJoin(condition=[IS NOT DISTINCT FROM($0, $1)], joinType=[semi]) :- LogicalProject(c=[$2]) : +- LogicalFilter(condition=[=(1, 0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -74,10 +74,10 @@ LogicalAggregate(group=[{0}]) @@ -85,10 +85,10 @@ LogicalIntersect(all=[false]) LogicalAggregate(group=[{0}]) +- LogicalJoin(condition=[IS NOT DISTINCT FROM($0, $1)], joinType=[semi]) :- LogicalProject(c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(f=[$2]) +- LogicalFilter(condition=[=(1, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -102,9 +102,9 @@ LogicalProject(c=[$2]) +- LogicalFilter(condition=[>($0, 1)]) +- LogicalIntersect(all=[false]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(d=[$0], e=[$1], f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -114,9 +114,9 @@ LogicalProject(c=[$2]) +- LogicalAggregate(group=[{0, 1, 2}]) +- LogicalJoin(condition=[AND(OR(=($0, $3), AND(IS NULL($0), IS NULL($3))), OR(=($1, $4), AND(IS NULL($1), IS NULL($4))), OR(=($2, $5), AND(IS NULL($2), IS NULL($5))))], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(d=[$0], e=[$1], f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ReplaceMinusWithAntiJoinRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ReplaceMinusWithAntiJoinRuleTest.xml index 3b81ba4ab7612..451ca7854084d 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ReplaceMinusWithAntiJoinRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ReplaceMinusWithAntiJoinRuleTest.xml @@ -24,9 +24,9 @@ limitations under the License. @@ -34,9 +34,9 @@ LogicalMinus(all=[false]) LogicalAggregate(group=[{0}]) +- LogicalJoin(condition=[IS NOT DISTINCT FROM($0, $1)], joinType=[anti]) :- LogicalProject(c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -49,9 +49,9 @@ LogicalAggregate(group=[{0}]) LogicalMinus(all=[false]) :- LogicalProject(c=[$2]) : +- LogicalFilter(condition=[=(1, 0)]) -: +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -60,9 +60,9 @@ LogicalAggregate(group=[{0}]) +- LogicalJoin(condition=[IS NOT DISTINCT FROM($0, $1)], joinType=[anti]) :- LogicalProject(c=[$2]) : +- LogicalFilter(condition=[=(1, 0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -74,10 +74,10 @@ LogicalAggregate(group=[{0}]) @@ -85,10 +85,10 @@ LogicalMinus(all=[false]) LogicalAggregate(group=[{0}]) +- LogicalJoin(condition=[IS NOT DISTINCT FROM($0, $1)], joinType=[anti]) :- LogicalProject(c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(f=[$2]) +- LogicalFilter(condition=[=(1, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -102,9 +102,9 @@ LogicalProject(c=[$2]) +- LogicalFilter(condition=[<($1, 2)]) +- LogicalMinus(all=[false]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(d=[$0], e=[$1], f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -114,9 +114,9 @@ LogicalProject(c=[$2]) +- LogicalAggregate(group=[{0, 1, 2}]) +- LogicalJoin(condition=[AND(OR(=($0, $3), AND(IS NULL($0), IS NULL($3))), OR(=($1, $4), AND(IS NULL($1), IS NULL($4))), OR(=($2, $5), AND(IS NULL($2), IS NULL($5))))], joinType=[anti]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(d=[$0], e=[$1], f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRuleTest.xml index ad832c4bab505..7a3862488a894 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRuleTest.xml @@ -24,9 +24,9 @@ limitations under the License. @@ -39,10 +39,10 @@ LogicalProject(c=[$2]) : +- LogicalUnion(all=[true]) : :- LogicalProject(c=[$0], vcol_left_marker=[true], vcol_right_marker=[null:BOOLEAN]) : : +- LogicalProject(c=[$2]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) : +- LogicalProject(f=[$0], vcol_left_marker=[null:BOOLEAN], vcol_right_marker=[true]) : +- LogicalProject(f=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType(VARCHAR(2147483647) c)]) ]]> @@ -56,9 +56,9 @@ LogicalProject(c=[$2]) LogicalIntersect(all=[true]) :- LogicalProject(c=[$2]) : +- LogicalFilter(condition=[=(1, 0)]) -: +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -72,10 +72,10 @@ LogicalProject(c=[$2]) : :- LogicalProject(c=[$0], vcol_left_marker=[true], vcol_right_marker=[null:BOOLEAN]) : : +- LogicalProject(c=[$2]) : : +- LogicalFilter(condition=[=(1, 0)]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) : +- LogicalProject(f=[$0], vcol_left_marker=[null:BOOLEAN], vcol_right_marker=[true]) : +- LogicalProject(f=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType(VARCHAR(2147483647) c)]) ]]> @@ -90,9 +90,9 @@ LogicalProject(c=[$2]) +- LogicalFilter(condition=[>($0, 1)]) +- LogicalIntersect(all=[true]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(d=[$0], e=[$1], f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -107,10 +107,10 @@ LogicalProject(c=[$2]) : +- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], b=[$1], c=[$2], vcol_left_marker=[true], vcol_right_marker=[null:BOOLEAN]) : : +- LogicalProject(a=[$0], b=[$1], c=[$2]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) : +- LogicalProject(d=[$0], e=[$1], f=[$2], vcol_left_marker=[null:BOOLEAN], vcol_right_marker=[true]) : +- LogicalProject(d=[$0], e=[$1], f=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1, $2, $3)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)]) ]]> @@ -123,10 +123,10 @@ LogicalProject(c=[$2]) @@ -139,11 +139,11 @@ LogicalProject(c=[$2]) : +- LogicalUnion(all=[true]) : :- LogicalProject(c=[$0], vcol_left_marker=[true], vcol_right_marker=[null:BOOLEAN]) : : +- LogicalProject(c=[$2]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) : +- LogicalProject(f=[$0], vcol_left_marker=[null:BOOLEAN], vcol_right_marker=[true]) : +- LogicalProject(f=[$2]) : +- LogicalFilter(condition=[=(1, 0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType(VARCHAR(2147483647) c)]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRuleTest.xml index e8956490d1cb6..3d200b48badff 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRuleTest.xml @@ -24,9 +24,9 @@ limitations under the License. @@ -39,10 +39,10 @@ LogicalProject(c=[$2]) : +- LogicalUnion(all=[true]) : :- LogicalProject(c=[$0], vcol_marker=[1:BIGINT]) : : +- LogicalProject(c=[$2]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) : +- LogicalProject(f=[$0], vcol_marker=[-1:BIGINT]) : +- LogicalProject(f=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType(VARCHAR(2147483647) c)]) ]]> @@ -56,9 +56,9 @@ LogicalProject(c=[$2]) LogicalMinus(all=[true]) :- LogicalProject(c=[$2]) : +- LogicalFilter(condition=[=(1, 0)]) -: +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -72,10 +72,10 @@ LogicalProject(c=[$2]) : :- LogicalProject(c=[$0], vcol_marker=[1:BIGINT]) : : +- LogicalProject(c=[$2]) : : +- LogicalFilter(condition=[=(1, 0)]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) : +- LogicalProject(f=[$0], vcol_marker=[-1:BIGINT]) : +- LogicalProject(f=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType(VARCHAR(2147483647) c)]) ]]> @@ -88,10 +88,10 @@ LogicalProject(c=[$2]) @@ -104,11 +104,11 @@ LogicalProject(c=[$2]) : +- LogicalUnion(all=[true]) : :- LogicalProject(c=[$0], vcol_marker=[1:BIGINT]) : : +- LogicalProject(c=[$2]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) : +- LogicalProject(f=[$0], vcol_marker=[-1:BIGINT]) : +- LogicalProject(f=[$2]) : +- LogicalFilter(condition=[=(1, 0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType(VARCHAR(2147483647) c)]) ]]> @@ -123,9 +123,9 @@ LogicalProject(c=[$2]) +- LogicalFilter(condition=[<($1, 2)]) +- LogicalMinus(all=[true]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(d=[$0], e=[$1], f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -140,10 +140,10 @@ LogicalProject(c=[$2]) : +- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], b=[$1], c=[$2], vcol_marker=[1:BIGINT]) : : +- LogicalProject(a=[$0], b=[$1], c=[$2]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) : +- LogicalProject(d=[$0], e=[$1], f=[$2], vcol_marker=[-1:BIGINT]) : +- LogicalProject(d=[$0], e=[$1], f=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1, $2, $3)], 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/rules/logical/RewriteMultiJoinConditionRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteMultiJoinConditionRuleTest.xml index 48dc828e00672..45ed377b6f072 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteMultiJoinConditionRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteMultiJoinConditionRuleTest.xml @@ -24,15 +24,15 @@ limitations under the License. @@ -45,18 +45,18 @@ MultiJoin(joinFilter=[=($0, $2)], isFullOuterJoin=[true], joinTypes=[[INNER, INN LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3], c1=[$4], c2=[$5]) +- LogicalJoin(condition=[=($0, $4)], joinType=[full]) :- LogicalJoin(condition=[=($0, $2)], joinType=[full]) - : :- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, C, source: [TestTableSource(c1, c2)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, A]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, B]]) + +- LogicalTableScan(table=[[default_catalog, default_database, C]]) ]]> @@ -69,15 +69,15 @@ MultiJoin(joinFilter=[=($0, $4)], isFullOuterJoin=[true], joinTypes=[[INNER, INN LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3]) +- LogicalFilter(condition=[=($0, $2)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, A]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -91,17 +91,17 @@ LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3], c1=[$4], c2=[$5]) +- LogicalFilter(condition=[AND(=($0, $2), =($0, $4))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, C, source: [TestTableSource(c1, c2)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, A]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, B]]) + +- LogicalTableScan(table=[[default_catalog, default_database, C]]) ]]> @@ -115,17 +115,17 @@ LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3], c1=[$4], c2=[$5]) +- LogicalFilter(condition=[AND(=($0, $2), =($0, $4))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, C, source: [TestTableSource(c1, c2)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, A]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, B]]) + +- LogicalTableScan(table=[[default_catalog, default_database, C]]) ]]> @@ -140,19 +140,19 @@ LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3], c1=[$4], c2=[$5], d1=[$6], d2 +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[true], joinType=[inner]) : :- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, C, source: [TestTableSource(c1, c2)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, D, source: [TestTableSource(d1, d2)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, A]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, B]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, C]]) + +- LogicalTableScan(table=[[default_catalog, default_database, D]]) ]]> @@ -166,17 +166,17 @@ LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3], c1=[$4], c2=[$5]) +- LogicalFilter(condition=[AND(=($0, $2), >($0, $4))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, C, source: [TestTableSource(c1, c2)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, A]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, B]]) + +- LogicalTableScan(table=[[default_catalog, default_database, C]]) ]]> ($0, $4), =($0, $2))], isFullOuterJoin=[false], joinTypes=[[INNER, INNER, INNER]], outerJoinConditions=[[NULL, NULL, NULL]], projFields=[[{0, 1}, {0, 1}, {0, 1}]]) -:- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) -:- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]]) -+- LogicalTableScan(table=[[default_catalog, default_database, C, source: [TestTableSource(c1, c2)]]]) +:- LogicalTableScan(table=[[default_catalog, default_database, A]]) +:- LogicalTableScan(table=[[default_catalog, default_database, B]]) ++- LogicalTableScan(table=[[default_catalog, default_database, C]]) ]]> @@ -190,17 +190,17 @@ LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3], c1=[$4], c2=[$5]) +- LogicalFilter(condition=[AND(=($0, $2), >($0, $4))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, C, source: [TestTableSource(c1, c2)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, A]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, B]]) + +- LogicalTableScan(table=[[default_catalog, default_database, C]]) ]]> ($0, $4), =($0, $2))], isFullOuterJoin=[false], joinTypes=[[INNER, INNER, INNER]], outerJoinConditions=[[NULL, NULL, NULL]], projFields=[[{0, 1}, {0, 1}, {0, 1}]]) -:- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) -:- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]]) -+- LogicalTableScan(table=[[default_catalog, default_database, C, source: [TestTableSource(c1, c2)]]]) +:- LogicalTableScan(table=[[default_catalog, default_database, A]]) +:- LogicalTableScan(table=[[default_catalog, default_database, B]]) ++- LogicalTableScan(table=[[default_catalog, default_database, C]]) ]]> @@ -214,17 +214,17 @@ LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3], c1=[$4], c2=[$5]) +- LogicalFilter(condition=[AND(=(+($0, 1), $2), =(+($0, 1), $4))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, C, source: [TestTableSource(c1, c2)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, A]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, B]]) + +- LogicalTableScan(table=[[default_catalog, default_database, C]]) ]]> @@ -236,15 +236,15 @@ MultiJoin(joinFilter=[AND(=(+($0, 1), $4), =(+($0, 1), $2), =($4, $2))], isFullO @@ -257,17 +257,17 @@ MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[INNER, LEFT]], LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3], c1=[$4], c2=[$5]) +- LogicalJoin(condition=[=($2, $4)], joinType=[left]) :- LogicalJoin(condition=[=($0, $2)], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, C, source: [TestTableSource(c1, c2)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, A]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, B]]) + +- LogicalTableScan(table=[[default_catalog, default_database, C]]) ]]> @@ -280,18 +280,18 @@ MultiJoin(joinFilter=[=($0, $2)], isFullOuterJoin=[false], joinTypes=[[INNER, IN LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3], c1=[$4], c2=[$5]) +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) :- LogicalJoin(condition=[=($0, $2)], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, C, source: [TestTableSource(c1, c2)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, A]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, B]]) + +- LogicalTableScan(table=[[default_catalog, default_database, C]]) ]]> @@ -303,15 +303,15 @@ MultiJoin(joinFilter=[=($0, $4)], isFullOuterJoin=[false], joinTypes=[[INNER, IN @@ -324,18 +324,18 @@ MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[RIGHT, INNER]] LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3], c1=[$4], c2=[$5]) +- LogicalJoin(condition=[=($2, $4)], joinType=[right]) :- LogicalJoin(condition=[=($0, $2)], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, C, source: [TestTableSource(c1, c2)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, A]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, B]]) + +- LogicalTableScan(table=[[default_catalog, default_database, C]]) ]]> @@ -348,18 +348,18 @@ MultiJoin(joinFilter=[true], isFullOuterJoin=[false], joinTypes=[[RIGHT, INNER]] LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3], c1=[$4], c2=[$5]) +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) :- LogicalJoin(condition=[=($0, $2)], joinType=[right]) - : :- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, C, source: [TestTableSource(c1, c2)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, A]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, B]]) + +- LogicalTableScan(table=[[default_catalog, default_database, C]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SimplifyFilterConditionRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SimplifyFilterConditionRuleTest.xml index e476f30e0d192..a4f54e8e5b531 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SimplifyFilterConditionRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SimplifyFilterConditionRuleTest.xml @@ -28,11 +28,11 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), =(2, 2), >($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalProject($f0=[0]) LogicalFilter(condition=[AND(=($0, 5), =($1, 6))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }), 0))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -43,11 +43,11 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), >($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalProject($f0=[0]) LogicalFilter(condition=[AND(=($0, 5), =($1, 6))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }), 0))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -63,11 +63,11 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), >($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalProject($f0=[0]) LogicalFilter(condition=[OR(AND(=($0, 5), =($1, 6)), AND(NOT(<>($0, 5)), =($1, 7), true))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }), 0))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -78,11 +78,11 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), >($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalProject($f0=[0]) LogicalFilter(condition=[AND(=($0, 5), SEARCH($1, Sarg[6L:BIGINT, 7L:BIGINT]:BIGINT))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }), 0))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -98,11 +98,11 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), =(2, 2), >($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalProject($f0=[0]) LogicalFilter(condition=[OR(AND(=($0, 5), =($1, 6)), AND(NOT(<>($0, 5)), =($1, 7), true))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }), 0))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -113,11 +113,11 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), >($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalProject($f0=[0]) LogicalFilter(condition=[AND(=($0, 5), SEARCH($1, Sarg[6L:BIGINT, 7L:BIGINT]:BIGINT))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }), 0))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -129,14 +129,14 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) ($0, 1)), =($2, 3), true))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -148,14 +148,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) @@ -167,14 +167,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) @@ -187,9 +187,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(EXISTS({ LogicalFilter(condition=[OR(AND(=($0, 1), =($1, 2)), AND(NOT(<>($0, 1)), =($1, 3)))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), true)]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -197,9 +197,9 @@ LogicalFilter(condition=[OR(AND(=($0, 1), =($1, 2)), AND(NOT(<>($0, 1)), =($1, 3 LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[AND(=($0, 1), SEARCH($1, Sarg[2L:BIGINT, 3L:BIGINT]:BIGINT))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -212,9 +212,9 @@ LogicalFilter(condition=[AND(=($0, 1), SEARCH($1, Sarg[2L:BIGINT, 3L:BIGINT]:BIG LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[OR(AND(=($0, 1), =($1, 2)), AND(NOT(<>($0, 1)), =($1, 3), true, EXISTS({ LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -222,9 +222,9 @@ LogicalFilter(condition=[>($0, 10)]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(=($0, 1), OR(=($1, 2), AND(=($1, 3), EXISTS({ LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -239,11 +239,11 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalFilter(condition=[AND(IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[OR(AND(=($0, 1), =($1, 2)), AND(NOT(<>($0, 1)), =($1, 3), true))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }), >($1, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -253,11 +253,11 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalFilter(condition=[AND(IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[AND(=($0, 1), SEARCH($1, Sarg[2L:BIGINT, 3L:BIGINT]:BIGINT))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }), >($1, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SimplifyJoinConditionRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SimplifyJoinConditionRuleTest.xml index ee6b40bda1b27..29dad4846ef2b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SimplifyJoinConditionRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SimplifyJoinConditionRuleTest.xml @@ -30,9 +30,9 @@ LogicalProject(a=[$0]) LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalProject($f0=[0]) LogicalFilter(condition=[OR(AND(=($0, $cor0.a), <($0, 2)), AND(=($0, $cor0.a), =($cor0.b, 5)))]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -40,14 +40,14 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalProject(a=[$0]) +- LogicalFilter(condition=[=($1, CASE(IS NULL($5), 0:BIGINT, $5))]) +- LogicalJoin(condition=[AND(=($0, $3), =($1, $4))], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[COUNT()]) +- LogicalProject(a=[$3], b=[$4]) +- LogicalJoin(condition=[AND(=($0, $3), OR(<($0, 2), =($4, 5)))], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) +- LogicalAggregate(group=[{0, 1}]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -59,16 +59,16 @@ LogicalProject(a=[$0]) ($0, 2)), AND(=($3, $0), =($1, 1)))], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> ($0, 2), =($1, 1)))], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -80,16 +80,16 @@ LogicalProject(d=[$3]) @@ -101,16 +101,16 @@ LogicalProject(d=[$3]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.xml index d054a354c043e..4137bad3f07dd 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.xml @@ -34,7 +34,7 @@ GROUP BY a ($1, 2))], $f3=[IS TRUE(<>($1, 5))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -43,7 +43,7 @@ FlinkLogicalCalc(select=[a, $f1, $f2, $f3, CAST(IF(=($f5, 0), null:INTEGER, /($f +- FlinkLogicalAggregate(group=[{0}], agg#0=[$SUM0($2)], agg#1=[SUM($3)], agg#2=[$SUM0($4)], agg#3=[$SUM0($5)], agg#4=[$SUM0($6)], agg#5=[SUM($7)]) +- FlinkLogicalAggregate(group=[{0, 4}], agg#0=[COUNT(DISTINCT $1) FILTER $2], agg#1=[SUM($1) FILTER $3], agg#2=[COUNT($1)], agg#3=[$SUM0($1)], agg#4=[COUNT($1)], agg#5=[SUM($1)]) +- FlinkLogicalCalc(select=[a, b, IS TRUE(<>(b, 2)) AS $f2, IS TRUE(<>(b, 5)) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -63,7 +63,7 @@ GROUP BY a ($1, 2))], $f3=[IS TRUE(<>($1, 5))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -71,7 +71,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1) FILTER $2], EXPR$2=[SUM FlinkLogicalAggregate(group=[{0}], agg#0=[$SUM0($2)], agg#1=[SUM($3)], agg#2=[SUM($4)]) +- FlinkLogicalAggregate(group=[{0, 4}], agg#0=[COUNT(DISTINCT $1) FILTER $2], agg#1=[SUM($1) FILTER $3], agg#2=[SUM($1) FILTER $2]) +- FlinkLogicalCalc(select=[a, b, IS TRUE(<>(b, 2)) AS $f2, IS TRUE(<>(b, 5)) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -101,8 +101,8 @@ LogicalProject(c=[$0], b=[$1], d=[$2], a=[$3], a0=[$4], b0=[$5], c0=[$6]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$2]) : +- LogicalAggregate(group=[{0}], b=[COUNT(DISTINCT $1)], d=[SUM($1)]) : +- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -115,8 +115,8 @@ FlinkLogicalJoin(condition=[=($1, $4)], joinType=[inner]) : +- FlinkLogicalAggregate(group=[{0}], agg#0=[$SUM0($2)], agg#1=[SUM($3)]) : +- FlinkLogicalAggregate(group=[{0, 2}], agg#0=[COUNT(DISTINCT $1)], agg#1=[SUM($1)]) : +- FlinkLogicalCalc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2]) -: +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -+- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ++- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -128,7 +128,7 @@ FlinkLogicalJoin(condition=[=($1, $4)], joinType=[inner]) @@ -136,7 +136,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT(DISTINCT $0)]) FlinkLogicalAggregate(group=[{}], agg#0=[$SUM0($1)]) +- FlinkLogicalAggregate(group=[{1}], agg#0=[COUNT(DISTINCT $0)]) +- FlinkLogicalCalc(select=[c, MOD(HASH_CODE(c), 100) AS $f1]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -148,7 +148,7 @@ FlinkLogicalAggregate(group=[{}], agg#0=[$SUM0($1)]) @@ -158,7 +158,7 @@ FlinkLogicalAggregate(group=[{}], agg#0=[$SUM0($2)], agg#1=[SUM($3)]) +- FlinkLogicalCalc(select=[a, b, $f2, $f3, =($e, 1) AS $g_1, =($e, 2) AS $g_2]) +- FlinkLogicalExpand(projects=[{a, b, $f2, null AS $f3, 1 AS $e}, {a, b, null AS $f2, $f3, 2 AS $e}]) +- FlinkLogicalCalc(select=[a, b, MOD(HASH_CODE(a), 1024) AS $f2, MOD(HASH_CODE(b), 1024) AS $f3]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -180,7 +180,7 @@ FROM( LogicalAggregate(group=[{0}], EXPR$1=[MIN($1)], EXPR$2=[MAX($1)], EXPR$3=[SUM($1)], EXPR$4=[COUNT()], EXPR$5=[COUNT(DISTINCT $2)]) +- LogicalProject(c=[$2], b=[$1], a=[$0]) +- LogicalAggregate(group=[{0}], b=[AVG($1)], c=[MAX($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -191,7 +191,7 @@ FlinkLogicalAggregate(group=[{0}], agg#0=[MIN($3)], agg#1=[MAX($4)], agg#2=[SUM( +- FlinkLogicalExpand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}, {a, b, c, null AS $f3, null AS $f4, 3 AS $e}]) +- FlinkLogicalCalc(select=[a, b, c, MOD(HASH_CODE(a), 1024) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) +- FlinkLogicalAggregate(group=[{0}], b=[AVG($1)], c=[MAX($2)]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -212,7 +212,7 @@ GROUP BY a ($1, 5))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -221,7 +221,7 @@ FlinkLogicalCalc(select=[a, $f1, $f2, $f3, CAST(IF(=($f5, 0), null:INTEGER, /($f +- FlinkLogicalAggregate(group=[{0}], agg#0=[$SUM0($2)], agg#1=[$SUM0($3)], agg#2=[SUM($4)], agg#3=[$SUM0($5)], agg#4=[$SUM0($6)]) +- FlinkLogicalAggregate(group=[{0, 3}], agg#0=[COUNT(DISTINCT $1)], agg#1=[COUNT(DISTINCT $1) FILTER $2], agg#2=[SUM($1)], agg#3=[$SUM0($1)], agg#4=[COUNT($1)]) +- FlinkLogicalCalc(select=[a, b, IS TRUE(<>(b, 5)) AS $f2, MOD(HASH_CODE(b), 1024) AS $f3]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -233,7 +233,7 @@ FlinkLogicalCalc(select=[a, $f1, $f2, $f3, CAST(IF(=($f5, 0), null:INTEGER, /($f @@ -241,7 +241,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT(DISTINCT $0)]) FlinkLogicalAggregate(group=[{}], agg#0=[$SUM0($1)]) +- FlinkLogicalAggregate(group=[{1}], agg#0=[COUNT(DISTINCT $0)]) +- FlinkLogicalCalc(select=[c, MOD(HASH_CODE(c), 1024) AS $f1]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -257,7 +257,7 @@ GROUP BY a @@ -268,7 +268,7 @@ FlinkLogicalCalc(select=[a, $f1 AS EXPR$1, CASE(=($f3, 0), null:INTEGER, $f2) AS +- FlinkLogicalCalc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 3) AS $g_3, =($e, 1) AS $g_1]) +- FlinkLogicalExpand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}, {a, b, c, null AS $f3, null AS $f4, 3 AS $e}]) +- FlinkLogicalCalc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -280,7 +280,7 @@ FlinkLogicalCalc(select=[a, $f1 AS EXPR$1, CASE(=($f3, 0), null:INTEGER, $f2) AS @@ -289,7 +289,7 @@ FlinkLogicalCalc(select=[a, $f1, $f2, CAST(IF(=($f4, 0), null:INTEGER, /($f3, $f +- FlinkLogicalAggregate(group=[{0}], agg#0=[$SUM0($2)], agg#1=[SUM($3)], agg#2=[$SUM0($4)], agg#3=[$SUM0($5)]) +- FlinkLogicalAggregate(group=[{0, 2}], agg#0=[COUNT(DISTINCT $1)], agg#1=[SUM($1)], agg#2=[$SUM0($1)], agg#3=[COUNT($1)]) +- FlinkLogicalCalc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -301,7 +301,7 @@ FlinkLogicalCalc(select=[a, $f1, $f2, CAST(IF(=($f4, 0), null:INTEGER, /($f3, $f @@ -309,7 +309,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)]) FlinkLogicalAggregate(group=[{0}], agg#0=[$SUM0($2)]) +- FlinkLogicalAggregate(group=[{0, 2}], agg#0=[COUNT(DISTINCT $1)]) +- FlinkLogicalCalc(select=[a, c, MOD(HASH_CODE(c), 1024) AS $f2]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -322,7 +322,7 @@ FlinkLogicalAggregate(group=[{0}], agg#0=[$SUM0($2)]) LogicalProject(EXPR$0=[$1]) +- LogicalAggregate(group=[{0}], EXPR$0=[FIRST_VALUE($1)]) +- LogicalProject(a=[$0], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -330,7 +330,7 @@ LogicalProject(EXPR$0=[$1]) FlinkLogicalCalc(select=[EXPR$0]) +- FlinkLogicalAggregate(group=[{0}], EXPR$0=[FIRST_VALUE($1)]) +- FlinkLogicalCalc(select=[a, c]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -342,7 +342,7 @@ FlinkLogicalCalc(select=[EXPR$0]) @@ -352,7 +352,7 @@ FlinkLogicalAggregate(group=[{0}], agg#0=[FIRST_VALUE($3)], agg#1=[$SUM0($4)]) +- FlinkLogicalCalc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 1) AS $g_1]) +- FlinkLogicalExpand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}]) +- FlinkLogicalCalc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -364,7 +364,7 @@ FlinkLogicalAggregate(group=[{0}], agg#0=[FIRST_VALUE($3)], agg#1=[$SUM0($4)]) @@ -374,7 +374,7 @@ FlinkLogicalAggregate(group=[{0}], agg#0=[LAST_VALUE($3)], agg#1=[$SUM0($4)]) +- FlinkLogicalCalc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 1) AS $g_1]) +- FlinkLogicalExpand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}]) +- FlinkLogicalCalc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -386,14 +386,14 @@ FlinkLogicalAggregate(group=[{0}], agg#0=[LAST_VALUE($3)], agg#1=[$SUM0($4)]) @@ -405,7 +405,7 @@ FlinkLogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) @@ -413,7 +413,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[LISTAGG($1)], EXPR$2=[COUNT(DISTINCT $2)]) FlinkLogicalAggregate(group=[{0}], agg#0=[LISTAGG($2)], agg#1=[$SUM0($3)]) +- FlinkLogicalAggregate(group=[{0, 3}], agg#0=[LISTAGG($2)], agg#1=[COUNT(DISTINCT $1)]) +- FlinkLogicalCalc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -428,7 +428,7 @@ GROUP BY a @@ -438,7 +438,7 @@ FlinkLogicalAggregate(group=[{0}], agg#0=[$SUM0($3)], agg#1=[MAX($4)]) +- FlinkLogicalCalc(select=[a, b, c, $f3, $f4, =($e, 1) AS $g_1, =($e, 2) AS $g_2]) +- FlinkLogicalExpand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}]) +- FlinkLogicalCalc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -450,7 +450,7 @@ FlinkLogicalAggregate(group=[{0}], agg#0=[$SUM0($3)], agg#1=[MAX($4)]) @@ -458,7 +458,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $0)], EXPR$2=[COUNT($1)]) FlinkLogicalAggregate(group=[{0}], agg#0=[$SUM0($1)], agg#1=[$SUM0($2)]) +- FlinkLogicalAggregate(group=[{0}], agg#0=[COUNT(DISTINCT $0)], agg#1=[COUNT($1)]) +- FlinkLogicalCalc(select=[a, b]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromCorrelateRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromCorrelateRuleTest.xml index 5fe73e79c78b0..674a48d231380 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromCorrelateRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromCorrelateRuleTest.xml @@ -25,7 +25,7 @@ limitations under the License. LogicalProject(a=[$0], b=[$1], c=[$2], s=[$3], l=[$4]) +- LogicalFilter(condition=[AND(=($4, $0), =($2, $3), =(pyFunc($4, $4), 2), =(+($4, 1), *($4, $4)))]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableFunctionScan(invocation=[func($cor0.c)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0, INTEGER f1)]) ]]> @@ -34,7 +34,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], s=[$3], l=[$4]) FlinkLogicalCalc(select=[a, b, c, f0, f1], where=[AND(=(f00, 2), =(f1, a), =(c, f0))]) +- FlinkLogicalCalc(select=[a, b, c, f0, f1, pyFunc(f1, f1) AS f00]) +- FlinkLogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - :- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + :- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) +- FlinkLogicalCalc(select=[f0, f1], where=[=(+(f1, 1), *(f1, f1))]) +- FlinkLogicalTableFunctionScan(invocation=[func($cor0.c)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0, INTEGER f1)]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRuleTest.xml index 4ba3979083251..b9363987891fb 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRuleTest.xml @@ -24,8 +24,8 @@ limitations under the License. @@ -34,9 +34,9 @@ FlinkLogicalCalc(select=[a, b, d], where=[=(f0, b)]) +- FlinkLogicalCalc(select=[a, b, d, pyFunc(a, d) AS f0]) +- FlinkLogicalJoin(condition=[=($0, $2)], joinType=[inner]) :- FlinkLogicalCalc(select=[a, b]) - : +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, leftTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, leftTable]], fields=[a, b, c]) +- FlinkLogicalCalc(select=[d]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, rightTable, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, rightTable]], fields=[d, e, f]) ]]> @@ -58,8 +58,8 @@ LogicalProject(a=[$0], EXPR$1=[+($1, 1)]) +- LogicalFilter(condition=[=(pyFunc($0, $1), *($0, $1))]) +- LogicalProject(a=[$0], d=[$3]) +- LogicalJoin(condition=[AND(=($0, $3), =(pyFunc($0, $0), +($0, $3)))], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, leftTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, rightTable, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, leftTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, rightTable]]) ]]> @@ -68,9 +68,9 @@ FlinkLogicalCalc(select=[a, +(d, 1) AS EXPR$1], where=[AND(=(f0, +(a, d)), =(f1, +- FlinkLogicalCalc(select=[a, d, pyFunc(a, a) AS f0, pyFunc(a, d) AS f1]) +- FlinkLogicalJoin(condition=[=($0, $1)], joinType=[inner]) :- FlinkLogicalCalc(select=[a]) - : +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, leftTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, leftTable]], fields=[a, b, c]) +- FlinkLogicalCalc(select=[d]) - +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, rightTable, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, rightTable]], fields=[d, e, f]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/WindowGroupReorderRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/WindowGroupReorderRuleTest.xml index 1ccfa68325a3a..03ff0534a5082 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/WindowGroupReorderRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/WindowGroupReorderRuleTest.xml @@ -31,14 +31,14 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST)], EXPR$2=[CAST(/(CASE(>(COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST), null:INTEGER), COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST))):INTEGER], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)], EXPR$4=[MIN($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST)]) -+- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> ($3, 0), $4, null:INTEGER)], EXPR$1=[$6], EXPR$2=[CAST(/(CASE(>($7, 0), $8, null:INTEGER), $7)):INTEGER], EXPR$3=[$5], EXPR$4=[$9]) +- LogicalWindow(window#0=[window(partition {1} order by [0 ASC-nulls-first] aggs [COUNT($0), $SUM0($0), RANK()])], window#1=[window(partition {1} order by [2 ASC-nulls-first] aggs [MAX($0)])], window#2=[window(partition {2} order by [0 ASC-nulls-first] aggs [COUNT($0), $SUM0($0), MIN($0)])]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -57,14 +57,14 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST)], EXPR$2=[MIN($0) OVER (ORDER BY $2 NULLS FIRST, $0 NULLS FIRST)], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST)], EXPR$4=[CAST(/(CASE(>(COUNT($0) OVER (ORDER BY $1 NULLS FIRST), 0), $SUM0($0) OVER (ORDER BY $1 NULLS FIRST), null:INTEGER), COUNT($0) OVER (ORDER BY $1 NULLS FIRST))):INTEGER]) -+- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> ($6, 0), $7, null:INTEGER)], EXPR$1=[$9], EXPR$2=[$5], EXPR$3=[$8], EXPR$4=[CAST(/(CASE(>($3, 0), $4, null:INTEGER), $3)):INTEGER]) +- LogicalWindow(window#0=[window(order by [1 ASC-nulls-first] aggs [COUNT($0), $SUM0($0)])], window#1=[window(order by [2 ASC-nulls-first, 0 ASC-nulls-first] aggs [MIN($0)])], window#2=[window(partition {1} order by [2 ASC-nulls-first] aggs [COUNT($0), $SUM0($0), RANK()])], window#3=[window(partition {2} order by [0 ASC-nulls-first] aggs [MAX($0)])]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -83,14 +83,14 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)], EXPR$2=[CAST(/(CASE(>(COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST), null:INTEGER), COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST))):INTEGER], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)], EXPR$4=[MIN($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST)]) -+- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> ($3, 0), $4, null:INTEGER)], EXPR$1=[$5], EXPR$2=[CAST(/(CASE(>($7, 0), $8, null:INTEGER), $7)):INTEGER], EXPR$3=[$6], EXPR$4=[$9]) +- LogicalWindow(window#0=[window(partition {1} order by [0 ASC-nulls-first] aggs [COUNT($0), $SUM0($0), MAX($0), RANK()])], window#1=[window(partition {2} order by [0 ASC-nulls-first] aggs [COUNT($0), $SUM0($0), MIN($0)])]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -109,7 +109,7 @@ SELECT (COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST), null:INTEGER)], EXPR$2=[RANK() OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST, $2 NULLS FIRST)], EXPR$3=[CASE(>(COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING), 0), $SUM0($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING), null:INTEGER)], EXPR$4=[COUNT() OVER (PARTITION BY $2 ORDER BY $2 NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING)]) -+- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -117,7 +117,7 @@ LogicalProject(EXPR$0=[COUNT() OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RAN LogicalProject(EXPR$0=[$2], EXPR$1=[CASE(>($3, 0), $4, null:INTEGER)], EXPR$2=[$7], EXPR$3=[CASE(>($5, 0), $6, null:INTEGER)], EXPR$4=[$8]) +- LogicalWindow(window#0=[window(partition {1} order by [0 ASC-nulls-first] range between $2 PRECEDING and $3 FOLLOWING aggs [COUNT()])], window#1=[window(partition {1} order by [0 ASC-nulls-first] aggs [COUNT($0), $SUM0($0)])], window#2=[window(partition {1} order by [0 ASC-nulls-first] range between $4 PRECEDING and $3 FOLLOWING aggs [COUNT($0), $SUM0($0)])], window#3=[window(partition {1} order by [0 ASC-nulls-first, 1 ASC-nulls-first] aggs [RANK()])], window#4=[window(partition {1} order by [1 ASC-nulls-first] rows between $4 PRECEDING and $3 FOLLOWING aggs [COUNT()])]) +- LogicalProject(a=[$0], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -136,14 +136,14 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $1 NULLS FIRST)], EXPR$2=[CAST(/(CASE(>(COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), null:INTEGER), COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST))):INTEGER], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST)], EXPR$4=[MIN($0) OVER (PARTITION BY $1 ORDER BY $1 NULLS FIRST)]) -+- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> ($7, 0), $8, null:INTEGER)], EXPR$1=[$5], EXPR$2=[CAST(/(CASE(>($3, 0), $4, null:INTEGER), $3)):INTEGER], EXPR$3=[$9], EXPR$4=[$6]) +- LogicalWindow(window#0=[window(partition {1} order by [0 ASC-nulls-first] aggs [COUNT($0), $SUM0($0)])], window#1=[window(partition {1} order by [1 ASC-nulls-first] aggs [MAX($0), MIN($0)])], window#2=[window(partition {1} order by [2 ASC-nulls-first] aggs [COUNT($0), $SUM0($0), RANK()])]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -162,14 +162,14 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)], EXPR$2=[CAST(/(CASE(>(COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST, $2 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST, $2 NULLS FIRST), null:INTEGER), COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST, $2 NULLS FIRST))):INTEGER], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST, $1 NULLS FIRST)], EXPR$4=[MIN($0) OVER (PARTITION BY $1 ORDER BY $1 NULLS FIRST)]) -+- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> ($8, 0), $9, null:INTEGER)], EXPR$1=[$3], EXPR$2=[CAST(/(CASE(>($5, 0), $6, null:INTEGER), $5)):INTEGER], EXPR$3=[$4], EXPR$4=[$7]) +- LogicalWindow(window#0=[window(partition {1} order by [0 ASC-nulls-first] aggs [MAX($0)])], window#1=[window(partition {1} order by [0 ASC-nulls-first, 1 ASC-nulls-first] aggs [RANK()])], window#2=[window(partition {1} order by [0 ASC-nulls-first, 2 ASC-nulls-first] aggs [COUNT($0), $SUM0($0)])], window#3=[window(partition {1} order by [1 ASC-nulls-first] aggs [MIN($0)])], window#4=[window(partition {1} order by [2 ASC-nulls-first] aggs [COUNT($0), $SUM0($0)])]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -188,7 +188,7 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)], EXPR$2=[CAST(/(CASE(>(COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 DESC NULLS LAST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 DESC NULLS LAST), null:INTEGER), COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 DESC NULLS LAST))):INTEGER], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)], EXPR$4=[MIN($0) OVER (PARTITION BY $1 ORDER BY $0 DESC NULLS LAST)]) -+- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -196,7 +196,7 @@ LogicalProject(EXPR$0=[CASE(>(COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS LogicalProject(EXPR$0=[CASE(>($2, 0), $3, null:INTEGER)], EXPR$1=[$4], EXPR$2=[CAST(/(CASE(>($6, 0), $7, null:INTEGER), $6)):INTEGER], EXPR$3=[$5], EXPR$4=[$8]) +- LogicalWindow(window#0=[window(partition {1} order by [0 ASC-nulls-first] aggs [COUNT($0), $SUM0($0), MAX($0), RANK()])], window#1=[window(partition {1} order by [0 DESC-nulls-last] aggs [COUNT($0), $SUM0($0), MIN($0)])]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -212,14 +212,14 @@ FROM MyTable @@ -236,7 +236,7 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST), null:INTEGER)], EXPR$1=[MIN($0) OVER (PARTITION BY $1)], EXPR$2=[MAX($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)]) -+- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -244,7 +244,7 @@ LogicalProject(EXPR$0=[CASE(>(COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS LogicalProject(EXPR$0=[CASE(>($3, 0), $4, null:INTEGER)], EXPR$1=[$2], EXPR$2=[$5]) +- LogicalWindow(window#0=[window(partition {1} aggs [MIN($0)])], window#1=[window(partition {1} order by [0 ASC-nulls-first] aggs [COUNT($0), $SUM0($0), MAX($0)])]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -260,14 +260,14 @@ FROM MyTable diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/FlinkRewriteSubQueryRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/FlinkRewriteSubQueryRuleTest.xml index 5e6b92c3fdbf3..3f288e654fbdc 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/FlinkRewriteSubQueryRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/FlinkRewriteSubQueryRuleTest.xml @@ -27,9 +27,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalAggregate(group=[{}], EXPR$0=[COUNT($0)]) LogicalProject(e=[$1]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -38,11 +38,11 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>($3, 0)]) +- LogicalJoin(condition=[true], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalAggregate(group=[{}], EXPR$0=[COUNT($0)]) +- LogicalProject(e=[$1]) +- LogicalFilter(condition=[>($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -58,9 +58,9 @@ LogicalProject(EXPR$0=[$1]) LogicalAggregate(group=[{0}], EXPR$0=[COUNT()]) LogicalProject(f=[$2]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -69,13 +69,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>($3, 0)]) +- LogicalJoin(condition=[true], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalAggregate(group=[{}], agg#0=[SINGLE_VALUE($0)]) +- LogicalProject(EXPR$0=[$1]) +- LogicalAggregate(group=[{0}], EXPR$0=[COUNT()]) +- LogicalProject(f=[$2]) +- LogicalFilter(condition=[>($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -141,9 +141,9 @@ LogicalSort(sort0=[$0], dir0=[ASC-nulls-first], fetch=[100]) +- LogicalFilter(condition=[AND(>=($0, 738), <=($0, +(738, 40)), >($SCALAR_QUERY({ LogicalAggregate(group=[{}], item_cnt=[COUNT()]) LogicalFilter(condition=[OR(AND(=($1, $cor0.i_manufact), OR(AND(=($3, _UTF-16LE'Women'), OR(=($4, _UTF-16LE'powder'), =($4, _UTF-16LE'khaki')), OR(=($5, _UTF-16LE'Ounce'), =($5, _UTF-16LE'Oz')), OR(=($6, _UTF-16LE'medium'), =($6, _UTF-16LE'extra large'))), AND(=($3, _UTF-16LE'Women'), OR(=($4, _UTF-16LE'brown'), =($4, _UTF-16LE'honeydew')), OR(=($5, _UTF-16LE'Bunch'), =($5, _UTF-16LE'Ton')), OR(=($6, _UTF-16LE'N/A'), =($6, _UTF-16LE'small'))), AND(=($3, _UTF-16LE'Men'), OR(=($4, _UTF-16LE'floral'), =($4, _UTF-16LE'deep')), OR(=($5, _UTF-16LE'N/A'), =($5, _UTF-16LE'Dozen')), OR(=($6, _UTF-16LE'petite'), =($6, _UTF-16LE'large'))), AND(=($3, _UTF-16LE'Men'), OR(=($4, _UTF-16LE'light'), =($4, _UTF-16LE'cornflower')), OR(=($5, _UTF-16LE'Box'), =($5, _UTF-16LE'Pound')), OR(=($6, _UTF-16LE'medium'), =($6, _UTF-16LE'extra large'))))), AND(=($1, $cor0.i_manufact), OR(AND(=($3, _UTF-16LE'Women'), OR(=($4, _UTF-16LE'midnight'), =($4, _UTF-16LE'snow')), OR(=($5, _UTF-16LE'Pallet'), =($5, _UTF-16LE'Gross')), OR(=($6, _UTF-16LE'medium'), =($6, _UTF-16LE'extra large'))), AND(=($3, _UTF-16LE'Women'), OR(=($4, _UTF-16LE'cyan'), =($4, _UTF-16LE'papaya')), OR(=($5, _UTF-16LE'Cup'), =($5, _UTF-16LE'Dram')), OR(=($6, _UTF-16LE'N/A'), =($6, _UTF-16LE'small'))), AND(=($3, _UTF-16LE'Men'), OR(=($4, _UTF-16LE'orange'), =($4, _UTF-16LE'frosted')), OR(=($5, _UTF-16LE'Each'), =($5, _UTF-16LE'Tbl')), OR(=($6, _UTF-16LE'petite'), =($6, _UTF-16LE'large'))), AND(=($3, _UTF-16LE'Men'), OR(=($4, _UTF-16LE'forest'), =($4, _UTF-16LE'ghost')), OR(=($5, _UTF-16LE'Lb'), =($5, _UTF-16LE'Bundle')), OR(=($6, _UTF-16LE'medium'), =($6, _UTF-16LE'extra large'))))))]) - LogicalTableScan(table=[[default_catalog, default_database, item, source: [TestTableSource(i_manufact_id, i_manufact, i_product_name, i_category, i_color, i_units, i_size)]]]) + LogicalTableScan(table=[[default_catalog, default_database, item]]) }), 0))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, item, source: [TestTableSource(i_manufact_id, i_manufact, i_product_name, i_category, i_color, i_units, i_size)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, item]]) ]]> @@ -153,10 +153,10 @@ LogicalSort(sort0=[$0], dir0=[ASC-nulls-first], fetch=[100]) +- LogicalProject(i_product_name=[$2]) +- LogicalFilter(condition=[AND(>=($0, 738), <=($0, +(738, 40)))]) +- LogicalJoin(condition=[=($7, $1)], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, item, source: [TestTableSource(i_manufact_id, i_manufact, i_product_name, i_category, i_color, i_units, i_size)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, item]]) +- LogicalProject(i_manufact=[$1]) +- LogicalFilter(condition=[OR(AND(=($3, _UTF-16LE'Women'), SEARCH($4, Sarg[_UTF-16LE'khaki':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'powder':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), SEARCH($5, Sarg[_UTF-16LE'Ounce':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Oz':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), SEARCH($6, Sarg[_UTF-16LE'extra large':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'medium':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE")), AND(=($3, _UTF-16LE'Women'), SEARCH($4, Sarg[_UTF-16LE'brown':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'honeydew':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), SEARCH($5, Sarg[_UTF-16LE'Bunch':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Ton':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), SEARCH($6, Sarg[_UTF-16LE'N/A':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'small':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE")), AND(=($3, _UTF-16LE'Men'), SEARCH($4, Sarg[_UTF-16LE'deep':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'floral':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), SEARCH($5, Sarg[_UTF-16LE'Dozen':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'N/A':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), SEARCH($6, Sarg[_UTF-16LE'large':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'petite':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE")), AND(=($3, _UTF-16LE'Men'), SEARCH($4, Sarg[_UTF-16LE'cornflower':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'light':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), SEARCH($5, Sarg[_UTF-16LE'Box':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Pound':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), SEARCH($6, Sarg[_UTF-16LE'extra large':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'medium':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE")), AND(=($3, _UTF-16LE'Women'), SEARCH($4, Sarg[_UTF-16LE'midnight':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'snow':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), SEARCH($5, Sarg[_UTF-16LE'Gross':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Pallet':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), SEARCH($6, Sarg[_UTF-16LE'extra large':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'medium':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE")), AND(=($3, _UTF-16LE'Women'), SEARCH($4, Sarg[_UTF-16LE'cyan':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'papaya':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), SEARCH($5, Sarg[_UTF-16LE'Cup':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Dram':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), SEARCH($6, Sarg[_UTF-16LE'N/A':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'small':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE")), AND(=($3, _UTF-16LE'Men'), SEARCH($4, Sarg[_UTF-16LE'frosted':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'orange':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), SEARCH($5, Sarg[_UTF-16LE'Each':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Tbl':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), SEARCH($6, Sarg[_UTF-16LE'large':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'petite':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE")), AND(=($3, _UTF-16LE'Men'), SEARCH($4, Sarg[_UTF-16LE'forest':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'ghost':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), SEARCH($5, Sarg[_UTF-16LE'Bundle':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Lb':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), SEARCH($6, Sarg[_UTF-16LE'extra large':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'medium':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]:VARCHAR(2147483647) CHARACTER SET "UTF-16LE")))]) - +- LogicalTableScan(table=[[default_catalog, default_database, item, source: [TestTableSource(i_manufact_id, i_manufact, i_product_name, i_category, i_color, i_units, i_size)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, item]]) ]]> @@ -170,19 +170,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), 0)], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -196,19 +196,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), 0.9:DECIMAL(2, 1))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -222,19 +222,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>=($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), 1)], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -248,19 +248,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>=($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), 0.1:DECIMAL(2, 1))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -274,19 +274,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[<(0, $SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -300,19 +300,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[<(0.99:DECIMAL(3, 2), $SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -326,19 +326,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[<=(1, $SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -352,19 +352,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[<=(0.01:DECIMAL(3, 2), $SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -378,21 +378,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> ($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -406,21 +406,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), 0.9:DECIMAL(2, 1))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> ($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -434,21 +434,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>=($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), 1)]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> ($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -462,21 +462,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>=($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), 0.1:DECIMAL(2, 1))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> ($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -490,21 +490,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[<(0, $SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> ($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -518,21 +518,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[<(0.99:DECIMAL(3, 2), $SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> ($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -546,21 +546,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[<=(1, $SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> ($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -574,21 +574,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[<=(0.01:DECIMAL(3, 2), $SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> ($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.xml index 8a66523e28cb4..61395048aa082 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.xml @@ -25,21 +25,21 @@ limitations under the License. LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), NOT(IN($1, { LogicalProject(i=[$0]) LogicalFilter(condition=[AND(=($cor0.c, $2), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), EXISTS({ LogicalAggregate(group=[{0}], EXPR$0=[COUNT($0)]) LogicalProject(l=[$0]) LogicalFilter(condition=[LIKE($2, _UTF-16LE'Test')]) - LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t2]]) }), NOT(EXISTS({ LogicalFilter(condition=[<>($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -49,22 +49,22 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) :- LogicalJoin(condition=[AND(OR(IS NULL($1), IS NULL($3), =($1, $3)), =($2, $4))], joinType=[anti]) : :- LogicalJoin(condition=[$3], joinType=[semi]) : : :- LogicalJoin(condition=[=($0, $3)], joinType=[semi]) - : : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : : : +- LogicalProject(d=[$0]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, r]]) : : +- LogicalProject($f0=[IS NOT NULL($0)]) : : +- LogicalAggregate(group=[{}], m=[MIN($0)]) : : +- LogicalProject(i=[true]) : : +- LogicalAggregate(group=[{0}], EXPR$0=[COUNT($0)]) : : +- LogicalProject(l=[$0]) : : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'Test')]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) : +- LogicalProject(i=[$0], k=[$2]) : +- LogicalFilter(condition=[>($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) +- LogicalProject(e=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -77,14 +77,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), NOT(EXISTS({ LogicalFilter(condition=[AND(=($cor0.b, $1), >(CAST($2):BIGINT, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), >=(CAST($2):BIGINT, 1), NOT(EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -94,14 +94,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($0, $6)], joinType=[anti]) :- LogicalJoin(condition=[=($1, $6)], joinType=[anti]) : :- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, r]]) : +- LogicalProject(j=[$1]) : +- LogicalFilter(condition=[>(CAST($2):BIGINT, 50)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 100)]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -114,14 +114,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), NOT(EXISTS({ LogicalFilter(condition=[AND(=($cor0.b, $1), >(CAST($2):BIGINT, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), >=(CAST($2):BIGINT, 1), NOT(EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -131,14 +131,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($0, $6)], joinType=[anti]) :- LogicalJoin(condition=[=($1, $6)], joinType=[anti]) : :- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, r]]) : +- LogicalProject(j=[$1]) : +- LogicalFilter(condition=[>(CAST($2):BIGINT, 50)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 100)]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -150,12 +150,12 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) @@ -163,14 +163,14 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[=($0, $3)], joinType=[anti]) :- LogicalJoin(condition=[$3], joinType=[anti]) - : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : +- LogicalProject($f0=[IS NOT NULL($0)]) : +- LogicalAggregate(group=[{}], m=[MIN($0)]) : +- LogicalProject(i=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, r]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 100)]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -183,12 +183,12 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(NOT(EXISTS({ LogicalFilter(condition=[<>($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })), NOT(EXISTS({ LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -196,15 +196,15 @@ LogicalFilter(condition=[<($1, 100)]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[$3], joinType=[anti]) :- LogicalJoin(condition=[<>($0, $3)], joinType=[anti]) - : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : +- LogicalProject(d=[$0]) : +- LogicalFilter(condition=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, r]]) +- LogicalProject($f0=[IS NOT NULL($0)]) +- LogicalAggregate(group=[{}], m=[MIN($0)]) +- LogicalProject(i=[true]) +- LogicalFilter(condition=[<($1, 100)]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -217,25 +217,25 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[AND(NOT(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) +LogicalTableScan(table=[[default_catalog, default_database, t]]) })), =($cor0.a, $0))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -249,26 +249,26 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[=($cor0.d, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -281,14 +281,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalFilter(condition=[AND(=($0, $2), NOT(EXISTS({ LogicalFilter(condition=[>($0, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })), >=($1, 1), NOT(EXISTS({ LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -298,18 +298,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalJoin(condition=[$4], joinType=[anti]) :- LogicalJoin(condition=[$4], joinType=[anti]) : :- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) : +- LogicalProject($f0=[IS NOT NULL($0)]) : +- LogicalAggregate(group=[{}], m=[MIN($0)]) : +- LogicalProject(i=[true]) : +- LogicalFilter(condition=[>($0, 50)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) +- LogicalProject($f0=[IS NOT NULL($0)]) +- LogicalAggregate(group=[{}], m=[MIN($0)]) +- LogicalProject(i=[true]) +- LogicalFilter(condition=[<($1, 100)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -323,21 +323,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[=($cor1.a, $0)]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) }))], variablesSet=[[$cor1]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -351,19 +351,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -376,9 +376,9 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[AND(NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })), >($0, 2))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -386,10 +386,10 @@ LogicalFilter(condition=[=($cor0.a, $0)]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[>($0, 2)]) +- LogicalJoin(condition=[=($0, $2)], joinType=[anti]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(c=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -402,19 +402,19 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -427,19 +427,19 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), >($cor0.b, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> ($1, 10))], joinType=[anti]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(c=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -452,11 +452,11 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0]) +- LogicalFilter(condition=[AND(=($1, $3), <($0, 10), <($2, 15), NOT(EXISTS({ LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + LogicalTableScan(table=[[default_catalog, default_database, x]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -465,11 +465,11 @@ LogicalProject(a=[$0]) +- LogicalFilter(condition=[AND(=($1, $3), <($0, 10), <($2, 15))]) +- LogicalJoin(condition=[=($1, $4)], joinType=[anti]) :- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(b=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -482,19 +482,19 @@ LogicalProject(a=[$0]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[<($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -507,19 +507,19 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[IS TRUE(NOT(EXISTS({ LogicalFilter(condition=[=($1, $cor0.b)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -532,19 +532,19 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[=($1, $cor0.b)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -557,19 +557,19 @@ LogicalProject(a=[$0]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[=(NOT(EXISTS({ LogicalFilter(condition=[=($1, $cor0.b)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })), true)], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -582,22 +582,22 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -610,20 +610,20 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) @@ -636,9 +636,9 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[AND(NOT(EXISTS({ LogicalFilter(condition=[>($0, 0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })), <($0, 100))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -646,12 +646,12 @@ LogicalFilter(condition=[>($0, 0)]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[<($0, 100)]) +- LogicalJoin(condition=[$2], joinType=[anti]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject($f0=[IS NOT NULL($0)]) +- LogicalAggregate(group=[{}], m=[MIN($0)]) +- LogicalProject(i=[true]) +- LogicalFilter(condition=[>($0, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -663,9 +663,9 @@ LogicalProject(a=[$0], b=[$1]) @@ -673,11 +673,11 @@ LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTabl LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[=($1, 10)]) +- LogicalJoin(condition=[$2], joinType=[anti]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject($f0=[IS NOT NULL($0)]) +- LogicalAggregate(group=[{}], m=[MIN($0)]) +- LogicalProject(i=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -689,20 +689,20 @@ LogicalProject(a=[$0], b=[$1]) @@ -714,20 +714,20 @@ LogicalProject(a=[$0], b=[$1]) @@ -739,20 +739,20 @@ LogicalProject(a=[$0]) @@ -765,13 +765,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })), NOT(IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -779,13 +779,13 @@ LogicalProject(i=[$0]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(OR(IS NULL($0), IS NULL($3), =($0, $3)), =($1, $4))], joinType=[anti]) :- LogicalJoin(condition=[=($0, $3)], joinType=[anti]) - : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : +- LogicalProject(d=[$0]) : +- LogicalFilter(condition=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, r]]) +- LogicalProject(i=[$0], j=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -798,12 +798,12 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(NOT(IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -811,12 +811,12 @@ LogicalFilter(condition=[=($cor0.a, $0)]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[=($0, $3)], joinType=[anti]) :- LogicalJoin(condition=[OR(IS NULL($1), IS NULL($3), =($1, $3))], joinType=[anti]) - : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : +- LogicalProject(j=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) +- LogicalProject(d=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -831,21 +831,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(f1=[$3]) LogicalFilter(condition=[=($cor1.a, $0)]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) }))], variablesSet=[[$cor1]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -860,19 +860,19 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(c=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -886,9 +886,9 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[AND(>($1, 1), NOT(IN($0, { LogicalProject(c=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -896,10 +896,10 @@ LogicalProject(c=[$0]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[>($1, 1)]) +- LogicalJoin(condition=[AND(OR(IS NULL($0), IS NULL($2), =($0, $2)), =($1, $3))], joinType=[anti]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(c=[$0], d=[$1]) +- LogicalFilter(condition=[>($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -913,9 +913,9 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[AND(>($1, 1), NOT(IN(+($0, 1), { LogicalProject(c=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -925,10 +925,10 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[>($1, 1)]) +- LogicalJoin(condition=[AND(OR(IS NULL($2), IS NULL($3), =($2, $3)), =($1, $4))], joinType=[anti]) :- LogicalProject(a=[$0], b=[$1], $f2=[+($0, 1)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(c=[$0], d=[$1]) +- LogicalFilter(condition=[>($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -943,20 +943,20 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(c1=[$1]) LogicalFilter(condition=[AND(=($cor0.b, $0), >($0, 10))]) LogicalProject(d1=[*($1, 2)], c1=[+($0, 1)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> ($0, 10)]) +- LogicalProject(d1=[*($1, 2)], c1=[+($0, 1)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -970,19 +970,19 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(c=[$0]) LogicalFilter(condition=[>($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> ($1, $3))], joinType=[anti]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(c=[$0], d=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -996,19 +996,19 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(c=[$0]) LogicalFilter(condition=[>($cor0.b, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> ($1, 10))], joinType=[anti]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(c=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1022,9 +1022,9 @@ LogicalProject(b=[$1]) +- LogicalFilter(condition=[NOT(IN(CASE(>($0, 10), 1, 2), { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1033,10 +1033,10 @@ LogicalProject(b=[$1]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(OR(IS NULL($4), =($3, $4)), =($0, $4))], joinType=[anti]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(>($0, 10), 1, 2)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(d=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1049,13 +1049,13 @@ LogicalProject(b=[$1]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[NOT(IN(CASE(IN($0, { LogicalProject(EXPR$0=[1]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) }), 1, 2), { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1066,16 +1066,16 @@ LogicalProject(b=[$1]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(AND(<>($3, 0), IS NOT NULL($6), IS NOT NULL($0)), 1, 2)]) : +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : :- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : : +- LogicalProject(c=[$0], ck=[$0]) : : +- LogicalAggregate(group=[{}], c=[COUNT()]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : +- LogicalAggregate(group=[{0, 1}]) : +- LogicalProject(EXPR$0=[1], i=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) +- LogicalProject(d=[$0], f=[$2]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1088,13 +1088,13 @@ LogicalProject(b=[$1]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[NOT(IN(CASE(NOT(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) })), 1, 2), { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1105,16 +1105,16 @@ LogicalProject(b=[$1]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(OR(=($3, 0), AND(IS NULL($6), >=($4, $3), IS NOT NULL($0))), 1, 2)]) : +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : :- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(i=[$0]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : +- LogicalAggregate(group=[{0, 1}]) : +- LogicalProject(i=[$0], i0=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) +- LogicalProject(d=[$0], f=[$2]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1127,16 +1127,16 @@ LogicalProject(b=[$1]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[NOT(IN(CASE(NOT(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) })), 1, NOT(IN($0, { LogicalProject(j=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t2]]) })), 2, 3), { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1149,22 +1149,22 @@ LogicalProject(b=[$1]) : :- LogicalJoin(condition=[true], joinType=[inner]) : : :- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : : : :- LogicalJoin(condition=[true], joinType=[inner]) - : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) - : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : : +- LogicalAggregate(group=[{0, 1}]) : : : +- LogicalProject(i=[$0], i0=[true]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$0]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) : +- LogicalAggregate(group=[{0, 1}]) : +- LogicalProject(j=[$0], i=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(d=[$0], f=[$2]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1177,16 +1177,16 @@ LogicalProject(b=[$1]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[NOT(IN($1, CASE(NOT(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) })), 1, NOT(IN($0, { LogicalProject(j=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t2]]) })), 2, 3), { LogicalProject(e=[$1], d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1199,22 +1199,22 @@ LogicalProject(b=[$1]) : :- LogicalJoin(condition=[true], joinType=[inner]) : : :- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : : : :- LogicalJoin(condition=[true], joinType=[inner]) - : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) - : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : : +- LogicalAggregate(group=[{0, 1}]) : : : +- LogicalProject(i=[$0], i0=[true]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$0]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) : +- LogicalAggregate(group=[{0, 1}]) : +- LogicalProject(j=[$0], i=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(e=[$1], d=[$0], f=[$2]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1228,9 +1228,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(EXPR$0=[CAST($1):INTEGER]) LogicalFilter(condition=[=(CAST($cor0.b):INTEGER, CAST($0):INTEGER)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1239,10 +1239,10 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(OR(IS NULL($0), IS NULL($4), =($0, $4)), =($3, $5))], joinType=[anti]) :- LogicalProject(a=[$0], b=[$1], c=[$2], b0=[CAST($1):INTEGER]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(EXPR$0=[CAST($1):INTEGER], d=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1256,9 +1256,9 @@ LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) +- LogicalFilter(condition=[NOT(IN(+($0, 10), { LogicalProject(EXPR$0=[+($0, 1)]) LogicalFilter(condition=[=(+($cor0.b, 1), $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1269,10 +1269,10 @@ LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) +- LogicalJoin(condition=[AND(OR(IS NULL($3), IS NULL($5), =($3, $5)), =($4, $6))], joinType=[anti]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[+($1, 1)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[+($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(EXPR$0=[+($0, 1)], e=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1286,20 +1286,20 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($2, { LogicalProject(EXPR$0=[TRIM(FLAG(BOTH), _UTF-16LE' ', $2)]) LogicalFilter(condition=[=($cor0.a, +($0, 1))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1313,9 +1313,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), LIKE($2, _UTF-16LE'abc'), NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1323,10 +1323,10 @@ LogicalProject(d=[$0]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), LIKE($2, _UTF-16LE'abc'))]) +- LogicalJoin(condition=[AND(OR(IS NULL($0), IS NULL($3), =($0, $3)), =($1, $4))], joinType=[anti]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(d=[$0], e=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1342,21 +1342,21 @@ LogicalProject(EXPR$0=[*($0, 5)]) LogicalAggregate(group=[{}], e=[SUM($0)]) LogicalProject(e=[$1]) LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1370,19 +1370,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, $2, { LogicalProject(d=[$0], f=[$2]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1396,9 +1396,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, SUBSTRING($2, 1, 5), { LogicalProject(d=[$0], EXPR$1=[TRIM(FLAG(BOTH), _UTF-16LE' ', $2)]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1407,10 +1407,10 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(OR(IS NULL($0), IS NULL($4), =($0, $4)), OR(IS NULL($3), IS NULL($5), =($3, $5)), =($1, $6))], joinType=[anti]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[SUBSTRING($2, 1, 5)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(d=[$0], EXPR$1=[TRIM(FLAG(BOTH), _UTF-16LE' ', $2)], e=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1424,9 +1424,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN(+($0, 10), SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[+($0, 100)], EXPR$1=[TRIM(FLAG(BOTH), _UTF-16LE' ', $2)]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1435,10 +1435,10 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(OR(IS NULL($3), IS NULL($5), =($3, $5)), OR(IS NULL($4), IS NULL($6), =($4, $6)), =($1, $7))], joinType=[anti]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[+($0, 10)], $f4=[SUBSTRING($2, 1, 5)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(EXPR$0=[+($0, 100)], EXPR$1=[TRIM(FLAG(BOTH), _UTF-16LE' ', $2)], e=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1452,12 +1452,12 @@ LogicalProject(a=[$0]) +- LogicalFilter(condition=[AND(NOT(IN($0, { LogicalProject(c=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })), NOT(IN($1, { LogicalProject(f=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1465,12 +1465,12 @@ LogicalProject(f=[$1]) LogicalProject(a=[$0]) +- LogicalJoin(condition=[OR(IS NULL($1), IS NULL($2), =($1, $2))], joinType=[anti]) :- LogicalJoin(condition=[AND(OR(IS NULL($0), IS NULL($2), =($0, $2)), =($1, $3))], joinType=[anti]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(c=[$0], d=[$1]) : +- LogicalFilter(condition=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(f=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -1483,13 +1483,13 @@ LogicalProject(a=[$0]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[AND(NOT(IN($0, { LogicalProject(c=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })), NOT(IN($1, { LogicalProject(e=[$0]) LogicalFilter(condition=[=($1, $cor0.b)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1497,12 +1497,12 @@ LogicalProject(e=[$0]) LogicalProject(a=[$0], b=[$1]) +- LogicalJoin(condition=[AND(OR(IS NULL($1), IS NULL($2), =($1, $2)), =($3, $1))], joinType=[anti]) :- LogicalJoin(condition=[OR(IS NULL($0), IS NULL($2), =($0, $2))], joinType=[anti]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(c=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(e=[$0], f=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -1517,23 +1517,23 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(c=[$0]) LogicalFilter(condition=[AND(NOT(IN($1, { LogicalProject(f=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })), =($cor0.b, $1))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1549,24 +1549,24 @@ LogicalProject(c=[$0]) LogicalFilter(condition=[NOT(IN($1, { LogicalProject(f=[$1]) LogicalFilter(condition=[=($0, $cor0.c)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))], variablesSet=[[$cor0]]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1580,20 +1580,20 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($2, { LogicalProject(f1=[$3]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1607,18 +1607,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(c=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1632,9 +1632,9 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[AND(NOT(IN($0, { LogicalProject(c=[$0]) LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })), >($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1642,10 +1642,10 @@ LogicalProject(c=[$0]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[>($1, 10)]) +- LogicalJoin(condition=[OR(IS NULL($0), IS NULL($2), =($0, $2))], joinType=[anti]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(c=[$0]) +- LogicalFilter(condition=[<($1, 100)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1658,9 +1658,9 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[NOT(IN(+($0, 1), { LogicalProject(c=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1669,9 +1669,9 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalProject(a=[$0], b=[$1]) +- LogicalJoin(condition=[OR(IS NULL($2), IS NULL($3), =($2, $3))], joinType=[anti]) :- LogicalProject(a=[$0], b=[$1], $f2=[+($0, 1)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(c=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1684,9 +1684,9 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[NOT(IN(*($0, $1), { LogicalProject(d=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1695,9 +1695,9 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalProject(a=[$0], b=[$1]) +- LogicalJoin(condition=[OR(IS NULL($2), IS NULL($3), =($2, $3))], joinType=[anti]) :- LogicalProject(a=[$0], b=[$1], $f2=[*($0, $1)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(d=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1710,9 +1710,9 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[NOT(IN(CAST($0):BIGINT, { LogicalProject(d=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1721,9 +1721,9 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalProject(a=[$0], b=[$1]) +- LogicalJoin(condition=[OR(IS NULL($2), IS NULL($3), =($2, $3))], joinType=[anti]) :- LogicalProject(a=[$0], b=[$1], a0=[CAST($0):BIGINT]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(d=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1737,19 +1737,19 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(c=[$0]) LogicalFilter(condition=[IS NOT NULL($0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1764,20 +1764,20 @@ LogicalProject(a=[$0]) LogicalAggregate(group=[{0}]) LogicalProject(a=[$0]) LogicalFilter(condition=[<($0, 3)]) - LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + LogicalTableScan(table=[[default_catalog, default_database, x]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1790,9 +1790,9 @@ LogicalProject(a=[$0]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[NOT(IN(CASE(>($0, 10), 1, 2), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1801,9 +1801,9 @@ LogicalProject(b=[$1]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[OR(IS NULL($4), =($3, $4))], joinType=[anti]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(>($0, 10), 1, 2)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1816,12 +1816,12 @@ LogicalProject(b=[$1]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[NOT(IN(CASE(NOT(IN($0, { LogicalProject(EXPR$0=[1]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) })), 1, 2), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1832,15 +1832,15 @@ LogicalProject(b=[$1]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(OR(=($3, 0), AND(IS NULL($6), >=($4, $3), IS NOT NULL($0))), 1, 2)]) : +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : :- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : : +- LogicalProject(c=[$0], ck=[$0]) : : +- LogicalAggregate(group=[{}], c=[COUNT()]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : +- LogicalAggregate(group=[{0, 1}]) : +- LogicalProject(EXPR$0=[1], i=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) +- LogicalProject(d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1853,12 +1853,12 @@ LogicalProject(b=[$1]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[NOT(IN(CASE(NOT(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) })), 1, 2), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1869,15 +1869,15 @@ LogicalProject(b=[$1]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(OR(=($3, 0), AND(IS NULL($6), >=($4, $3), IS NOT NULL($0))), 1, 2)]) : +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : :- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(i=[$0]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : +- LogicalAggregate(group=[{0, 1}]) : +- LogicalProject(i=[$0], i0=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) +- LogicalProject(d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1890,15 +1890,15 @@ LogicalProject(b=[$1]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[NOT(IN(CASE(NOT(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) })), 1, NOT(IN($0, { LogicalProject(j=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t2]]) })), 2, 3), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1911,21 +1911,21 @@ LogicalProject(b=[$1]) : :- LogicalJoin(condition=[true], joinType=[inner]) : : :- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : : : :- LogicalJoin(condition=[true], joinType=[inner]) - : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) - : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : : +- LogicalAggregate(group=[{0, 1}]) : : : +- LogicalProject(i=[$0], i0=[true]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$0]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) : +- LogicalAggregate(group=[{0, 1}]) : +- LogicalProject(j=[$0], i=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1938,15 +1938,15 @@ LogicalProject(b=[$1]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[NOT(IN($1, CASE(NOT(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) })), 1, NOT(IN($0, { LogicalProject(j=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t2]]) })), 2, 3), { LogicalProject(e=[$1], d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1959,21 +1959,21 @@ LogicalProject(b=[$1]) : :- LogicalJoin(condition=[true], joinType=[inner]) : : :- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : : : :- LogicalJoin(condition=[true], joinType=[inner]) - : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) - : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : : +- LogicalAggregate(group=[{0, 1}]) : : : +- LogicalProject(i=[$0], i0=[true]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$0]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) : +- LogicalAggregate(group=[{0, 1}]) : +- LogicalProject(j=[$0], i=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(e=[$1], d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1986,15 +1986,15 @@ LogicalProject(b=[$1]) LogicalProject(c=[$2]) +- LogicalFilter(condition=[NOT(IN(CASE(NOT(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) })), 1, 2), CASE(NOT(IN($1, { LogicalProject(j=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t2]]) })), 3, 4), { LogicalProject(d=[$0], e=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2007,21 +2007,21 @@ LogicalProject(c=[$2]) : :- LogicalJoin(condition=[true], joinType=[inner]) : : :- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : : : :- LogicalJoin(condition=[true], joinType=[inner]) - : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) - : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : : +- LogicalAggregate(group=[{0, 1}]) : : : +- LogicalProject(i=[$0], i0=[true]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$0]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) : +- LogicalAggregate(group=[{0, 1}]) : +- LogicalProject(j=[$0], i=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -2034,18 +2034,18 @@ LogicalProject(c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2058,18 +2058,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2082,9 +2082,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })), NOT(>($1, 10)))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2092,9 +2092,9 @@ LogicalProject(d=[$0]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[<=($1, 10)]) +- LogicalJoin(condition=[OR(IS NULL($0), IS NULL($3), =($0, $3))], joinType=[anti]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -2107,9 +2107,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), LIKE($2, _UTF-16LE'abc'), NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2117,9 +2117,9 @@ LogicalProject(d=[$0]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), LIKE($2, _UTF-16LE'abc'))]) +- LogicalJoin(condition=[OR(IS NULL($0), IS NULL($3), =($0, $3))], joinType=[anti]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -2132,18 +2132,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[IS TRUE(NOT(IN($0, { LogicalProject(c=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -2156,18 +2156,18 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[=(NOT(IN($0, { LogicalProject(c=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })), true)]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -2180,18 +2180,18 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, $2, { LogicalProject(d=[$0], f=[$2]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2204,9 +2204,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN(+($0, 1), $2, { LogicalProject(d=[$0], f=[$2]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2215,9 +2215,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(OR(IS NULL($3), IS NULL($4), =($3, $4)), OR(IS NULL($2), IS NULL($5), =($2, $5)))], joinType=[anti]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[+($0, 1)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(d=[$0], f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -2230,9 +2230,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN(+($0, 10), TRIM(FLAG(BOTH), _UTF-16LE' ', $2), { LogicalProject(EXPR$0=[+($0, 100)], EXPR$1=[SUBSTRING($2, 1, 5)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2241,9 +2241,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(OR(IS NULL($3), IS NULL($5), =($3, $5)), OR(IS NULL($4), IS NULL($6), =($4, $6)))], joinType=[anti]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[+($0, 10)], $f4=[TRIM(FLAG(BOTH), _UTF-16LE' ', $2)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(EXPR$0=[+($0, 100)], EXPR$1=[SUBSTRING($2, 1, 5)]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- 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/logical/subquery/SubQuerySemiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.xml index 55d1aa8decbf5..831d380b7af38 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.xml @@ -26,21 +26,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=($cor1.a, $0)]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })], variablesSet=[[$cor1]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -54,19 +54,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -79,9 +79,9 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[AND(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), >($0, 2))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -89,10 +89,10 @@ LogicalFilter(condition=[=($cor0.a, $0)]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[>($0, 2)]) +- LogicalJoin(condition=[=($0, $2)], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(c=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -105,19 +105,19 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -130,19 +130,19 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), >($cor0.b, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> ($1, 10))], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(c=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -155,11 +155,11 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0]) +- LogicalFilter(condition=[AND(=($1, $3), <($0, 10), <($2, 15), EXISTS({ LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + LogicalTableScan(table=[[default_catalog, default_database, x]]) }))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -168,11 +168,11 @@ LogicalProject(a=[$0]) +- LogicalFilter(condition=[AND(=($1, $3), <($0, 10), <($2, 15))]) +- LogicalJoin(condition=[=($1, $4)], joinType=[semi]) :- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(b=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -185,19 +185,19 @@ LogicalProject(a=[$0]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[<($cor0.a, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -210,19 +210,19 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[IS TRUE(EXISTS({ LogicalFilter(condition=[=($1, $cor0.b)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -235,19 +235,19 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=($1, $cor0.b)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -262,21 +262,21 @@ LogicalProject(a=[$0], b=[$1]) LogicalAggregate(group=[{0}], EXPR$0=[COUNT()]) LogicalProject(f=[$3]) LogicalFilter(condition=[AND(=($cor0.b, $1), <($0, 100))]) - LogicalTableScan(table=[[default_catalog, default_database, r1, source: [TestTableSource(c, d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r1]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l1, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l1]]) ]]> @@ -291,21 +291,21 @@ LogicalProject(a=[$0], b=[$1]) LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$0=[MAX($4)]) LogicalProject(c=[$0], $f1=[true], f=[$3], $f3=[1], e=[$2]) LogicalFilter(condition=[AND(=($cor0.b, $1), <($0, 100), =($cor0.a, $0))]) - LogicalTableScan(table=[[default_catalog, default_database, r1, source: [TestTableSource(c, d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r1]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l1, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l1]]) ]]> @@ -321,22 +321,22 @@ LogicalFilter(condition=[>($cor0.c, $1)]) LogicalProject(d=[$1], f=[$0]) LogicalAggregate(group=[{0}], d=[MAX($1)]) LogicalProject(f=[$2], d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> ($2, $3)], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(f=[$1]) +- LogicalFilter(condition=[true]) +- LogicalProject(d=[$1], f=[$0]) +- LogicalAggregate(group=[{0}], d=[MAX($1)]) +- LogicalProject(f=[$2], d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -351,21 +351,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalAggregate(group=[{0}]) LogicalProject(e=[$1]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -378,9 +378,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=(CAST($cor0.b):INTEGER, CAST($0):INTEGER)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -389,10 +389,10 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[=($3, $4)], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], c=[$2], b0=[CAST($1):INTEGER]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(d=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -405,9 +405,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), NOT(OR(LIKE($2, _UTF-16LE'abc'), NOT(EXISTS({ LogicalFilter(condition=[=(+($cor0.b, 1), $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })))))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -417,10 +417,10 @@ LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[=($3, $4)], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[+($1, 1)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(e=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -433,19 +433,19 @@ LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[=(EXISTS({ LogicalFilter(condition=[=($1, $cor0.b)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), true)], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -460,23 +460,23 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalJoin(condition=[=($0, $3)], joinType=[inner]) LogicalProject(f=[$2]) LogicalFilter(condition=[AND(=($0, $cor0.a), >($1, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> ($1, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, r]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -489,25 +489,25 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalJoin(condition=[=($2, $5)], joinType=[inner]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0], j=[$1], k=[$2]) LogicalFilter(condition=[AND(=($1, $cor0.b), <($0, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -522,27 +522,27 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalJoin(condition=[=($0, $2)], joinType=[inner]) LogicalProject(f=[$2]) LogicalFilter(condition=[AND(=($0, $cor0.a), >($1, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0], k=[$2]) LogicalFilter(condition=[AND(=($1, $cor0.b), <($0, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> ($1, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, r]]) +- LogicalProject(i=[$0], k=[$2], j=[$1]) +- LogicalFilter(condition=[<($0, 50)]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -557,20 +557,20 @@ LogicalProject(a=[$0], b=[$1]) LogicalFilter(condition=[=($1, $cor1.b)]) LogicalProject(c=[$0], d=[$1]) LogicalFilter(condition=[=($0, $cor1.a)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })], variablesSet=[[$cor1]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -583,7 +583,7 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=($0, $cor0.a)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })], variablesSet=[[$cor0]]) +- LogicalProject(EXPR$0=[$0], EXPR$1=[$1]) +- LogicalValues(tuples=[[{ 1, 2 }]]) @@ -597,7 +597,7 @@ LogicalProject(a=[$0], b=[$1]) : +- LogicalValues(tuples=[[{ 1, 2 }]]) +- LogicalProject(c=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -612,23 +612,23 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalJoin(condition=[=($0, $3)], joinType=[left]) LogicalProject(f=[$2]) LogicalFilter(condition=[AND(=($0, $cor0.a), >($1, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> ($1, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, r]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -643,21 +643,21 @@ LogicalProject(a=[$0], b=[$1]) LogicalSort(fetch=[1]) LogicalProject(EXPR$0=[1]) LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -672,20 +672,20 @@ LogicalProject(a=[$0]) LogicalFilter(condition=[=($cor0.a, $0)]) LogicalSort(fetch=[1]) LogicalProject(c=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -698,19 +698,19 @@ LogicalProject(a=[$0]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[AND(=($cor0.c, $2), <($0, 3))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -723,19 +723,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[AND(<($cor0.c, $2), <($0, 3))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -748,25 +748,25 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalJoin(condition=[=($2, $5)], joinType=[right]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0], j=[$1], k=[$2]) LogicalFilter(condition=[AND(=($1, $cor0.b), <($0, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -781,21 +781,21 @@ LogicalProject(a=[$0], b=[$1]) LogicalSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[1]) LogicalProject(EXPR$0=[1], d=[$1]) LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -810,21 +810,21 @@ LogicalProject(a=[$0], b=[$1]) LogicalSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[1]) LogicalProject(EXPR$0=[1], c=[$0]) LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -837,22 +837,22 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -866,31 +866,31 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) +LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -905,30 +905,30 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) LogicalProject(d=[$0], e=[$1], f=[$2]) LogicalFilter(condition=[EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) +LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -942,22 +942,22 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -970,20 +970,20 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) @@ -996,12 +996,12 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1], d=[$2]) +- LogicalFilter(condition=[AND(EXISTS({ LogicalFilter(condition=[>($0, 0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), <($0, 100))]) +- LogicalProject(a=[$0], b=[$1], d=[$3]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1011,13 +1011,13 @@ LogicalProject(a=[$0], b=[$1], d=[$2]) +- LogicalJoin(condition=[$3], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], d=[$3]) : +- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject($f0=[IS NOT NULL($0)]) +- LogicalAggregate(group=[{}], m=[MIN($0)]) +- LogicalProject(i=[true]) +- LogicalFilter(condition=[>($0, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1029,9 +1029,9 @@ LogicalProject(a=[$0], b=[$1], d=[$2]) @@ -1039,11 +1039,11 @@ LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTabl LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[=($1, 10)]) +- LogicalJoin(condition=[$2], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject($f0=[IS NOT NULL($0)]) +- LogicalAggregate(group=[{}], m=[MIN($0)]) +- LogicalProject(i=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1055,20 +1055,20 @@ LogicalProject(a=[$0], b=[$1]) @@ -1080,20 +1080,20 @@ LogicalProject(a=[$0], b=[$1]) @@ -1106,9 +1106,9 @@ LogicalProject(a=[$0]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), EXISTS({ LogicalFilter(condition=[>($1, 0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1116,12 +1116,12 @@ LogicalFilter(condition=[>($1, 0)]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>($1, 10)]) +- LogicalJoin(condition=[$3], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject($f0=[IS NOT NULL($0)]) +- LogicalAggregate(group=[{}], m=[MIN($0)]) +- LogicalProject(i=[true]) +- LogicalFilter(condition=[>($1, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1133,9 +1133,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) ($1, 10), NOT(OR(LIKE($2, _UTF-16LE'abc'), NOT(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) +LogicalTableScan(table=[[default_catalog, default_database, r]]) })))))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1143,11 +1143,11 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), NOT(LIKE($2, _UTF-16LE'abc')))]) +- LogicalJoin(condition=[$3], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject($f0=[IS NOT NULL($0)]) +- LogicalAggregate(group=[{}], m=[MIN($0)]) +- LogicalProject(i=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1160,27 +1160,27 @@ LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) LogicalProject(c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalJoin(condition=[=($1, $3)], joinType=[full]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> ($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -1195,23 +1195,23 @@ LogicalProject(a=[$0]) LogicalAggregate(group=[{0}], EXPR$0=[MAX($1)], EXPR$1=[MIN($2)]) LogicalProject(f=[$2], e=[$1], d=[$0]) LogicalFilter(condition=[<($0, 3)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1226,29 +1226,29 @@ LogicalProject(a=[$0]) LogicalUnion(all=[false]) LogicalProject(e=[$1], f=[$2]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1], k=[$2]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> ($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, r]]) +- LogicalProject(j=[$1], k=[$2]) +- LogicalFilter(condition=[<($0, 100)]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -1260,20 +1260,20 @@ LogicalProject(a=[$0]) @@ -1286,13 +1286,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1300,13 +1300,13 @@ LogicalProject(i=[$0]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(=($0, $3), =($1, $4))], joinType=[semi]) :- LogicalJoin(condition=[=($0, $3)], joinType=[semi]) - : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : +- LogicalProject(d=[$0]) : +- LogicalFilter(condition=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, r]]) +- LogicalProject(i=[$0], j=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -1319,12 +1319,12 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1332,12 +1332,12 @@ LogicalFilter(condition=[=($cor0.a, $0)]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[=($0, $3)], joinType=[semi]) :- LogicalJoin(condition=[=($1, $3)], joinType=[semi]) - : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : +- LogicalProject(j=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) +- LogicalProject(d=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1352,21 +1352,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(f1=[$3]) LogicalFilter(condition=[=($cor1.a, $0)]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })], variablesSet=[[$cor1]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1381,19 +1381,19 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[IN($0, { LogicalProject(c=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1407,9 +1407,9 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[AND(>($1, 1), IN($0, { LogicalProject(c=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1417,10 +1417,10 @@ LogicalProject(c=[$0]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[>($1, 1)]) +- LogicalJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(c=[$0], d=[$1]) +- LogicalFilter(condition=[>($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1434,9 +1434,9 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[AND(>($1, 1), IN(+($0, 1), { LogicalProject(c=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1446,10 +1446,10 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[>($1, 1)]) +- LogicalJoin(condition=[AND(=($2, $3), =($1, $4))], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], $f2=[+($0, 1)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(c=[$0], d=[$1]) +- LogicalFilter(condition=[>($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1463,19 +1463,19 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[IN($0, { LogicalProject(c=[$0]) LogicalFilter(condition=[>($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> ($1, $3))], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(c=[$0], d=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1490,21 +1490,21 @@ LogicalProject(a=[$0]) LogicalAggregate(group=[{0}]) LogicalProject(a=[$0]) LogicalFilter(condition=[AND(<($0, 3), =($cor0.b, $1))]) - LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + LogicalTableScan(table=[[default_catalog, default_database, x]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1518,9 +1518,9 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[IN($0, { LogicalProject(c=[$0]) LogicalFilter(condition=[=(CAST($cor0.b):INTEGER, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -1529,10 +1529,10 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalProject(a=[$0], b=[$1]) +- LogicalJoin(condition=[AND(=($0, $3), =($2, $3))], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], b0=[CAST($1):INTEGER]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(c=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1546,19 +1546,19 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[IN($0, { LogicalProject(c=[$0]) LogicalFilter(condition=[>($cor0.b, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> ($1, 10))], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(c=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -1574,21 +1574,21 @@ LogicalProject(EXPR$0=[$1]) LogicalAggregate(group=[{0}], EXPR$0=[MAX($1)]) LogicalProject(f=[$2], e=[$1]) LogicalFilter(condition=[AND(=($cor0.c, $2), <($0, 3))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1604,21 +1604,21 @@ LogicalProject(EXPR$0=[*($0, 5)]) LogicalAggregate(group=[{}], d=[SUM($0)]) LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1632,21 +1632,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($1, { LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1661,21 +1661,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalAggregate(group=[{0}], EXPR$1=[COUNT()]) LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1691,21 +1691,21 @@ LogicalProject(EXPR$0=[$3]) LogicalAggregate(group=[{0, 1, 2}], EXPR$0=[MAX($3)]) LogicalProject(d=[$0], $f1=[true], $f2=[1], e=[$1]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1721,21 +1721,21 @@ LogicalProject(EXPR$0=[$4], d=[$0]) LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$0=[COUNT()]) LogicalProject(d=[$0], $f1=[true], e=[$1], $f3=[1]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1751,22 +1751,22 @@ LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) LogicalProject(e=[$1]) LogicalFilter(condition=[AND(<($2, 2), =($cor0.a, $0))]) LogicalProject(d=[$0], e=[$1], rk=[RANK() OVER (PARTITION BY $0 ORDER BY $1 NULLS FIRST)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1781,21 +1781,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalAggregate(group=[{}], EXPR$0=[AVG($0)]) LogicalProject(e=[$1]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1812,23 +1812,23 @@ LogicalProject(d=[$0]) LogicalProject(d=[$1], f=[$0]) LogicalAggregate(group=[{0}], d=[MAX($1)]) LogicalProject(f=[$2], d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> ($2, $4))], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(d=[$0], f=[$1], d0=[CAST($0):BIGINT]) +- LogicalProject(d=[$0], f=[$1]) +- LogicalFilter(condition=[true]) +- LogicalProject(d=[$1], f=[$0]) +- LogicalAggregate(group=[{0}], d=[MAX($1)]) +- LogicalProject(f=[$2], d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1844,21 +1844,21 @@ LogicalProject(EXPR$0=[MAX($0) OVER ()]) LogicalAggregate(group=[{0}]) LogicalProject(e=[$1]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1872,9 +1872,9 @@ LogicalProject(b=[$1]) +- LogicalFilter(condition=[IN(CASE(>($0, 10), 1, 2), { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1883,10 +1883,10 @@ LogicalProject(b=[$1]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(=($3, $4), =($0, $4))], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(>($0, 10), 1, 2)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(d=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1899,13 +1899,13 @@ LogicalProject(b=[$1]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[IN(CASE(IN($0, { LogicalProject(EXPR$0=[1]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i, j)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) }), 1, 2), { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1916,16 +1916,16 @@ LogicalProject(b=[$1]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(AND(<>($3, 0), IS NOT NULL($6), IS NOT NULL($0)), 1, 2)]) : +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : :- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : : +- LogicalProject(c=[$0], ck=[$0]) : : +- LogicalAggregate(group=[{}], c=[COUNT()]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i, j)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : +- LogicalAggregate(group=[{0, 1}]) : +- LogicalProject(EXPR$0=[1], i=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i, j)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) +- LogicalProject(d=[$0], f=[$2]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1938,13 +1938,13 @@ LogicalProject(b=[$1]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[IN(CASE(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i, j)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) }), 1, 2), { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1955,16 +1955,16 @@ LogicalProject(b=[$1]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(AND(<>($3, 0), IS NOT NULL($6), IS NOT NULL($0)), 1, 2)]) : +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : :- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(i=[$0]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i, j)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : +- LogicalAggregate(group=[{0, 1}]) : +- LogicalProject(i=[$0], i0=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i, j)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) +- LogicalProject(d=[$0], f=[$2]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1977,16 +1977,16 @@ LogicalProject(b=[$1]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[IN(CASE(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i, j)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) }), 1, IN($0, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(i, j)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t2]]) }), 2, 3), { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1999,22 +1999,22 @@ LogicalProject(b=[$1]) : :- LogicalJoin(condition=[true], joinType=[inner]) : : :- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : : : :- LogicalJoin(condition=[true], joinType=[inner]) - : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) - : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i, j)]]]) + : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : : +- LogicalAggregate(group=[{0, 1}]) : : : +- LogicalProject(i=[$0], i0=[true]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i, j)]]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$1]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(i, j)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) : +- LogicalAggregate(group=[{0, 1}]) : +- LogicalProject(j=[$1], i=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(i, j)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(d=[$0], f=[$2]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -2027,16 +2027,16 @@ LogicalProject(b=[$1]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[IN($1, CASE(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i, j)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) }), 1, IN($0, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(i, j)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t2]]) }), 2, 3), { LogicalProject(e=[$1], d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2049,22 +2049,22 @@ LogicalProject(b=[$1]) : :- LogicalJoin(condition=[true], joinType=[inner]) : : :- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : : : :- LogicalJoin(condition=[true], joinType=[inner]) - : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) - : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i, j)]]]) + : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : : +- LogicalAggregate(group=[{0, 1}]) : : : +- LogicalProject(i=[$0], i0=[true]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i, j)]]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$1]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(i, j)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) : +- LogicalAggregate(group=[{0, 1}]) : +- LogicalProject(j=[$1], i=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(i, j)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(e=[$1], d=[$0], f=[$2]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -2078,9 +2078,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(EXPR$0=[CAST($1):INTEGER]) LogicalFilter(condition=[=(CAST($cor0.b):INTEGER, CAST($0):INTEGER)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2089,10 +2089,10 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(=($0, $4), =($3, $5))], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], c=[$2], b0=[CAST($1):INTEGER]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(EXPR$0=[CAST($1):INTEGER], d=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -2106,9 +2106,9 @@ LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) +- LogicalFilter(condition=[IN(SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=(+($cor0.b, 1), $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2119,10 +2119,10 @@ LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) +- LogicalJoin(condition=[AND(=($3, $5), =($4, $6))], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[+($1, 1)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[SUBSTRING($2, 1, 5)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)], e=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -2136,9 +2136,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), NOT(LIKE($2, _UTF-16LE'abc')), IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2146,10 +2146,10 @@ LogicalProject(d=[$0]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), NOT(LIKE($2, _UTF-16LE'abc')))]) +- LogicalJoin(condition=[AND(=($0, $3), =($1, $4))], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(d=[$0], e=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -2163,9 +2163,9 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[IS TRUE(IN($0, { LogicalProject(d=[$1]) LogicalFilter(condition=[=($1, $cor0.b)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -2174,10 +2174,10 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalProject(a=[$0], b=[$1]) +- LogicalJoin(condition=[AND(=($2, $3), =($3, $1))], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], a0=[CAST($0):BIGINT]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(d=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -2191,26 +2191,26 @@ LogicalProject(c=[$2]) +- LogicalFilter(condition=[IN($1, { LogicalProject(d=[$0]) LogicalJoin(condition=[=($1, $3)], joinType=[inner]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1]) LogicalFilter(condition=[AND(=($cor0.c, $2), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> ($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -2226,23 +2226,23 @@ LogicalProject(j=[$3]) LogicalJoin(condition=[=($1, $4)], joinType=[inner]) LogicalProject(e=[$1], f=[$2]) LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2258,27 +2258,27 @@ LogicalProject(e=[$0]) LogicalJoin(condition=[=($0, $1)], joinType=[inner]) LogicalProject(e=[$1]) LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1]) LogicalFilter(condition=[AND(=($cor0.c, $2), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> ($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -2294,23 +2294,23 @@ LogicalProject(j=[$3]) LogicalJoin(condition=[=($1, $4)], joinType=[left]) LogicalProject(e=[$1], f=[$2]) LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2324,19 +2324,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, $2, { LogicalProject(d=[$0], f=[$2]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2350,9 +2350,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, SUBSTRING($2, 1, 5), { LogicalProject(d=[$0], EXPR$1=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2361,10 +2361,10 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(=($0, $4), =($3, $5), =($1, $6))], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[SUBSTRING($2, 1, 5)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(d=[$0], EXPR$1=[SUBSTRING($2, 1, 5)], e=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -2378,9 +2378,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN(+($0, 10), SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[+($0, 100)], EXPR$1=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2389,10 +2389,10 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(=($3, $5), =($4, $6), =($1, $7))], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[+($0, 10)], $f4=[SUBSTRING($2, 1, 5)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(EXPR$0=[+($0, 100)], EXPR$1=[SUBSTRING($2, 1, 5)], e=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -2406,19 +2406,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($1, { LogicalProject(EXPR$0=[MAX($1) OVER ()]) LogicalFilter(condition=[AND(=($cor0.c, $2), <($0, 3))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2432,19 +2432,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, $1, { LogicalProject(EXPR$0=[MAX($0) OVER ()], EXPR$1=[MIN($1) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST)]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2459,20 +2459,20 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(rk=[$2]) LogicalFilter(condition=[<>($cor0.a, $0)]) LogicalProject(d=[$0], e=[$1], rk=[RANK() OVER (PARTITION BY $0 ORDER BY $1 NULLS FIRST)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> ($0, $4))], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(rk=[$2], d=[$0]) +- LogicalFilter(condition=[true]) +- LogicalProject(d=[$0], e=[$1], rk=[RANK() OVER (PARTITION BY $0 ORDER BY $1 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -2486,26 +2486,26 @@ LogicalProject(c=[$2]) +- LogicalFilter(condition=[IN($1, { LogicalProject(d=[$0]) LogicalJoin(condition=[=($1, $3)], joinType=[right]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1]) LogicalFilter(condition=[AND(=($cor0.c, $2), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> ($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -2520,13 +2520,13 @@ LogicalProject(a=[$0]) LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) LogicalProject(d=[$1]) LogicalFilter(condition=[>($0, 0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), { LogicalProject(f=[$1]) LogicalFilter(condition=[=($0, $cor0.a)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -2535,14 +2535,14 @@ LogicalProject(a=[$0]) +- LogicalProject(a=[$0], b=[$1]) +- LogicalJoin(condition=[AND(=($2, $3), =($4, $0))], joinType=[semi]) :- LogicalJoin(condition=[true], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) : +- LogicalProject(d=[$1]) : +- LogicalFilter(condition=[>($0, 0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(f=[$1], e=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -2558,15 +2558,15 @@ LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) LogicalProject(d=[$1]) LogicalFilter(condition=[IN($0, { LogicalProject(e=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), { LogicalProject(f=[$1]) LogicalFilter(condition=[=($0, $cor0.a)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -2575,18 +2575,18 @@ LogicalProject(a=[$0]) +- LogicalProject(a=[$0], b=[$1]) +- LogicalJoin(condition=[AND(=($2, $3), =($4, $0))], joinType=[semi]) :- LogicalJoin(condition=[true], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) : +- LogicalProject(d=[$1]) : +- LogicalProject(c=[$0], d=[$1]) : +- LogicalJoin(condition=[=($0, $2)], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, y]]) : +- LogicalAggregate(group=[{0}]) : +- LogicalProject(e=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) +- LogicalProject(f=[$1], e=[$0]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -2603,15 +2603,15 @@ LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) LogicalFilter(condition=[IN($0, { LogicalProject(e=[$0]) LogicalFilter(condition=[=($cor1.d, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })], variablesSet=[[$cor1]]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), { LogicalProject(f=[$1]) LogicalFilter(condition=[=($0, $cor0.a)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -2619,17 +2619,17 @@ LogicalProject(f=[$1]) LogicalProject(a=[$0]) +- LogicalJoin(condition=[AND(=($2, $3), =($4, $0))], joinType=[semi]) :- LogicalJoin(condition=[true], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) : +- LogicalProject(d=[$1]) : +- LogicalJoin(condition=[AND(=($1, $3), =($0, $2))], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, y]]) : +- LogicalAggregate(group=[{0, 1}]) : +- LogicalProject(e=[$0], f=[$1]) : +- LogicalFilter(condition=[IS NOT NULL($1)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) +- LogicalProject(f=[$1], e=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -2643,15 +2643,15 @@ LogicalProject(b=[$1]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(c=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), >($1, $SCALAR_QUERY({ LogicalProject(EXPR$0=[*(0.5:DECIMAL(2, 1), $0)]) LogicalAggregate(group=[{}], agg#0=[SUM($0)]) LogicalProject(e=[$0]) LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -2661,16 +2661,16 @@ LogicalProject(b=[$1]) +- LogicalFilter(condition=[>($1, $2)]) +- LogicalJoin(condition=[true], joinType=[left]) :- LogicalJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[semi]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(c=[$0], d=[$1]) : +- LogicalFilter(condition=[>($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalAggregate(group=[{}], agg#0=[SINGLE_VALUE($0)]) +- LogicalProject(EXPR$0=[*(0.5:DECIMAL(2, 1), $0)]) +- LogicalAggregate(group=[{}], agg#0=[SUM($0)]) +- LogicalProject(e=[$0]) +- LogicalFilter(condition=[<($1, 100)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -2684,15 +2684,15 @@ LogicalProject(b=[$1]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(c=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), >($1, $SCALAR_QUERY({ LogicalProject(EXPR$0=[*(0.5:DECIMAL(2, 1), $0)]) LogicalAggregate(group=[{}], agg#0=[SUM($0)]) LogicalProject(e=[$0]) LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -2700,14 +2700,14 @@ LogicalProject(EXPR$0=[*(0.5:DECIMAL(2, 1), $0)]) LogicalProject(b=[$1]) +- LogicalJoin(condition=[AND(=($0, $2), >($1, *(0.5:DECIMAL(2, 1), $3)))], joinType=[inner]) :- LogicalJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[semi]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(c=[$0], d=[$1]) : +- LogicalFilter(condition=[>($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalAggregate(group=[{0}], agg#0=[SUM($1)]) +- LogicalProject(e1=[$0], e=[$0]) +- LogicalFilter(condition=[AND(<($1, 100), IS NOT NULL($0))]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -2724,25 +2724,25 @@ LogicalProject(c=[$0]) LogicalAggregate(group=[{}], EXPR$0=[SUM($0)]) LogicalProject(e=[$0]) LogicalFilter(condition=[AND(=($cor1.c, $0), <($1, 100))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))], variablesSet=[[$cor1]]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> ($0, $3))], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalAggregate(group=[{0}], EXPR$0=[SUM($1)]) +- LogicalProject(e1=[$0], e=[$0]) +- LogicalFilter(condition=[AND(<($1, 100), IS NOT NULL($0))]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -2755,11 +2755,11 @@ LogicalProject(b=[$1]) LogicalProject(s=[$1]) +- LogicalFilter(condition=[AND(>($2, 2), IN($0, { LogicalProject(d=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))]) +- LogicalAggregate(group=[{0}], s=[SUM($1)], agg#1=[COUNT()]) +- LogicalProject(b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -2769,9 +2769,9 @@ LogicalProject(s=[$1]) +- LogicalJoin(condition=[=($0, $3)], joinType=[semi]) :- LogicalAggregate(group=[{0}], s=[SUM($1)], agg#1=[COUNT()]) : +- LogicalProject(b=[$1], a=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(d=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -2784,11 +2784,11 @@ LogicalProject(s=[$1]) LogicalProject(s=[$1]) +- LogicalFilter(condition=[AND(>($2, 2), IN($3, { LogicalProject(d=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))]) +- LogicalAggregate(group=[{0}], s=[SUM($1)], agg#1=[COUNT()], agg#2=[MAX($0)]) +- LogicalProject(b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -2798,9 +2798,9 @@ LogicalProject(s=[$1]) +- LogicalJoin(condition=[=($3, $4)], joinType=[semi]) :- LogicalAggregate(group=[{0}], s=[SUM($1)], agg#1=[COUNT()], agg#2=[MAX($0)]) : +- LogicalProject(b=[$1], a=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(d=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -2814,20 +2814,20 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($2, { LogicalProject(f1=[$3]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2843,27 +2843,27 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(f1=[$3]) LogicalFilter(condition=[IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2880,26 +2880,26 @@ LogicalProject(f1=[$3]) LogicalProject(d=[$0], e=[$1], f=[$2]) LogicalFilter(condition=[IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2914,20 +2914,20 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($2, { LogicalProject(f1=[$3]) LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2941,18 +2941,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[IN($0, { LogicalProject(c=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -2966,9 +2966,9 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(c=[$0]) LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), >($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -2976,10 +2976,10 @@ LogicalProject(c=[$0]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[>($1, 10)]) +- LogicalJoin(condition=[=($0, $2)], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(c=[$0]) +- LogicalFilter(condition=[<($1, 100)]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -2992,9 +2992,9 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[IN(+($0, 1), { LogicalProject(c=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -3003,9 +3003,9 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalProject(a=[$0], b=[$1]) +- LogicalJoin(condition=[=($2, $3)], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], $f2=[+($0, 1)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(c=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -3018,9 +3018,9 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[IN(*($0, $1), { LogicalProject(d=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -3029,9 +3029,9 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalProject(a=[$0], b=[$1]) +- LogicalJoin(condition=[=($2, $3)], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], $f2=[*($0, $1)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(d=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -3044,9 +3044,9 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[IN(CAST($0):BIGINT, { LogicalProject(d=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -3055,9 +3055,9 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalProject(a=[$0], b=[$1]) +- LogicalJoin(condition=[=($2, $3)], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], a0=[CAST($0):BIGINT]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(d=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -3072,20 +3072,20 @@ LogicalProject(a=[$0]) LogicalAggregate(group=[{0}]) LogicalProject(a=[$0]) LogicalFilter(condition=[<($0, 3)]) - LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + LogicalTableScan(table=[[default_catalog, default_database, x]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -3101,21 +3101,21 @@ LogicalProject(EXPR$0=[$1]) LogicalAggregate(group=[{0}], EXPR$0=[MAX($1)]) LogicalProject(f=[$2], e=[$1]) LogicalFilter(condition=[<($0, 3)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3130,20 +3130,20 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(EXPR$0=[*($0, 5)]) LogicalAggregate(group=[{}], d=[SUM($0)]) LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3156,18 +3156,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($1, { LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3181,19 +3181,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, $1, { LogicalAggregate(group=[{0}], EXPR$1=[COUNT()]) LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3208,20 +3208,20 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(EXPR$0=[$3]) LogicalAggregate(group=[{0, 1, 2}], EXPR$0=[MAX($3)]) LogicalProject(d=[$0], $f1=[true], $f2=[1], e=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3236,20 +3236,20 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(EXPR$0=[$4], d=[$0]) LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$0=[COUNT()]) LogicalProject(d=[$0], $f1=[true], e=[$1], $f3=[1]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3264,20 +3264,20 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(EXPR$0=[MAX($0) OVER ()]) LogicalAggregate(group=[{0}]) LogicalProject(e=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3290,9 +3290,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[IN(CASE(>($0, 10), 1, 2), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3301,9 +3301,9 @@ LogicalProject(b=[$1]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[=($3, $4)], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(>($0, 10), 1, 2)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -3316,12 +3316,12 @@ LogicalProject(b=[$1]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[IN(CASE(IN($0, { LogicalProject(EXPR$0=[1]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) }), 1, 2), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3332,15 +3332,15 @@ LogicalProject(b=[$1]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(AND(<>($3, 0), IS NOT NULL($6), IS NOT NULL($0)), 1, 2)]) : +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : :- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : : +- LogicalProject(c=[$0], ck=[$0]) : : +- LogicalAggregate(group=[{}], c=[COUNT()]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : +- LogicalAggregate(group=[{0, 1}]) : +- LogicalProject(EXPR$0=[1], i=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) +- LogicalProject(d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -3353,12 +3353,12 @@ LogicalProject(b=[$1]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[IN(CASE(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) }), 1, 2), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3369,15 +3369,15 @@ LogicalProject(b=[$1]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(AND(<>($3, 0), IS NOT NULL($6), IS NOT NULL($0)), 1, 2)]) : +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : :- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(i=[$0]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : +- LogicalAggregate(group=[{0, 1}]) : +- LogicalProject(i=[$0], i0=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) +- LogicalProject(d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -3390,15 +3390,15 @@ LogicalProject(b=[$1]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[IN(CASE(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) }), 1, IN($0, { LogicalProject(j=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t2]]) }), 2, 3), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3411,21 +3411,21 @@ LogicalProject(b=[$1]) : :- LogicalJoin(condition=[true], joinType=[inner]) : : :- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : : : :- LogicalJoin(condition=[true], joinType=[inner]) - : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) - : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : : +- LogicalAggregate(group=[{0, 1}]) : : : +- LogicalProject(i=[$0], i0=[true]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$0]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) : +- LogicalAggregate(group=[{0, 1}]) : +- LogicalProject(j=[$0], i=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -3438,15 +3438,15 @@ LogicalProject(b=[$1]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[IN($1, CASE(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) }), 1, IN($0, { LogicalProject(j=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t2]]) }), 2, 3), { LogicalProject(e=[$1], d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3459,21 +3459,21 @@ LogicalProject(b=[$1]) : :- LogicalJoin(condition=[true], joinType=[inner]) : : :- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : : : :- LogicalJoin(condition=[true], joinType=[inner]) - : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) - : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : : +- LogicalAggregate(group=[{0, 1}]) : : : +- LogicalProject(i=[$0], i0=[true]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$0]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) : +- LogicalAggregate(group=[{0, 1}]) : +- LogicalProject(j=[$0], i=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(e=[$1], d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -3486,15 +3486,15 @@ LogicalProject(b=[$1]) LogicalProject(c=[$2]) +- LogicalFilter(condition=[IN(CASE(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) }), 1, 2), CASE(IN($1, { LogicalProject(j=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t2]]) }), 3, 4), { LogicalProject(d=[$0], e=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3509,21 +3509,21 @@ LogicalProject(c=[$2]) : :- LogicalJoin(condition=[true], joinType=[inner]) : : :- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : : : :- LogicalJoin(condition=[true], joinType=[inner]) - : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : : : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) - : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : : +- LogicalAggregate(group=[{0, 1}]) : : : +- LogicalProject(i=[$0], i0=[true]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$0]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) : +- LogicalAggregate(group=[{0, 1}]) : +- LogicalProject(j=[$0], i=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -3536,9 +3536,9 @@ LogicalProject(c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), NOT(LIKE($2, _UTF-16LE'abc')), IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3546,9 +3546,9 @@ LogicalProject(d=[$0]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), NOT(LIKE($2, _UTF-16LE'abc')))]) +- LogicalJoin(condition=[=($0, $3)], joinType=[semi]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(d=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -3562,26 +3562,26 @@ LogicalProject(c=[$2]) +- LogicalFilter(condition=[IN($1, { LogicalProject(d=[$0]) LogicalJoin(condition=[=($1, $3)], joinType=[full]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> ($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -3597,21 +3597,21 @@ LogicalProject(EXPR$0=[$1]) LogicalAggregate(group=[{0}], EXPR$0=[MAX($1)]) LogicalProject(f=[$2], e=[$1]) LogicalFilter(condition=[<($0, 3)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3626,26 +3626,26 @@ LogicalProject(a=[$0]) LogicalUnion(all=[false]) LogicalProject(e=[$1]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> ($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, r]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($0, 100)]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -3658,9 +3658,9 @@ LogicalProject(a=[$0]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[IS TRUE(IN($0, { LogicalProject(d=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -3669,9 +3669,9 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalProject(a=[$0], b=[$1]) +- LogicalJoin(condition=[=($2, $3)], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], a0=[CAST($0):BIGINT]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(d=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -3684,18 +3684,18 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, $2, { LogicalProject(d=[$0], f=[$2]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3708,9 +3708,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN(+($0, 1), $2, { LogicalProject(d=[$0], f=[$2]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3719,9 +3719,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(=($3, $4), =($2, $5))], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[+($0, 1)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(d=[$0], f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -3734,9 +3734,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN(+($0, 10), SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[+($0, 100)], EXPR$1=[SUBSTRING($2, 1, 5)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3745,9 +3745,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(=($3, $5), =($4, $6))], joinType=[semi]) :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[+($0, 10)], $f4=[SUBSTRING($2, 1, 5)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, l]]) +- LogicalProject(EXPR$0=[+($0, 100)], EXPR$1=[SUBSTRING($2, 1, 5)]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -3760,18 +3760,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($1, { LogicalProject(EXPR$0=[MAX($1) OVER ()]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3784,18 +3784,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, $1, { LogicalProject(EXPR$0=[MAX($0) OVER ()], EXPR$1=[MIN($1) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -3810,12 +3810,12 @@ LogicalProject(a=[$0]) LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) LogicalProject(d=[$1]) LogicalFilter(condition=[>($0, 0)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), { LogicalProject(f=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -3824,13 +3824,13 @@ LogicalProject(a=[$0]) +- LogicalProject(a=[$0], b=[$1]) +- LogicalJoin(condition=[=($2, $3)], joinType=[semi]) :- LogicalJoin(condition=[true], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) : +- LogicalProject(d=[$1]) : +- LogicalFilter(condition=[>($0, 0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(f=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -3846,14 +3846,14 @@ LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) LogicalProject(d=[$1]) LogicalFilter(condition=[IN($0, { LogicalProject(e=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), { LogicalProject(f=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -3862,17 +3862,17 @@ LogicalProject(a=[$0]) +- LogicalProject(a=[$0], b=[$1]) +- LogicalJoin(condition=[=($2, $3)], joinType=[semi]) :- LogicalJoin(condition=[true], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) : +- LogicalProject(d=[$1]) : +- LogicalProject(c=[$0], d=[$1]) : +- LogicalJoin(condition=[=($0, $2)], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, y]]) : +- LogicalAggregate(group=[{0}]) : +- LogicalProject(e=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) +- LogicalProject(f=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -3886,15 +3886,15 @@ LogicalProject(b=[$1]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(c=[$0]) LogicalFilter(condition=[>($1, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), >($1, $SCALAR_QUERY({ LogicalProject(EXPR$0=[*(0.5:DECIMAL(2, 1), $0)]) LogicalAggregate(group=[{}], agg#0=[SUM($0)]) LogicalProject(e=[$0]) LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -3904,16 +3904,16 @@ LogicalProject(b=[$1]) +- LogicalFilter(condition=[>($1, $2)]) +- LogicalJoin(condition=[true], joinType=[left]) :- LogicalJoin(condition=[=($0, $2)], joinType=[semi]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(c=[$0]) : +- LogicalFilter(condition=[>($1, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalAggregate(group=[{}], agg#0=[SINGLE_VALUE($0)]) +- LogicalProject(EXPR$0=[*(0.5:DECIMAL(2, 1), $0)]) +- LogicalAggregate(group=[{}], agg#0=[SUM($0)]) +- LogicalProject(e=[$0]) +- LogicalFilter(condition=[<($1, 100)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -3927,15 +3927,15 @@ LogicalProject(b=[$1]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(c=[$0]) LogicalFilter(condition=[>($1, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), >($1, $SCALAR_QUERY({ LogicalProject(EXPR$0=[*(0.5:DECIMAL(2, 1), $0)]) LogicalAggregate(group=[{}], agg#0=[SUM($0)]) LogicalProject(e=[$0]) LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -3943,14 +3943,14 @@ LogicalProject(EXPR$0=[*(0.5:DECIMAL(2, 1), $0)]) LogicalProject(b=[$1]) +- LogicalJoin(condition=[AND(=($0, $2), >($1, *(0.5:DECIMAL(2, 1), $3)))], joinType=[inner]) :- LogicalJoin(condition=[=($0, $2)], joinType=[semi]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(c=[$0]) : +- LogicalFilter(condition=[>($1, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalAggregate(group=[{0}], agg#0=[SUM($1)]) +- LogicalProject(e1=[$0], e=[$0]) +- LogicalFilter(condition=[AND(<($1, 100), IS NOT NULL($0))]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -3967,25 +3967,25 @@ LogicalProject(c=[$0]) LogicalAggregate(group=[{}], EXPR$0=[SUM($0)]) LogicalProject(e=[$0]) LogicalFilter(condition=[AND(=($cor0.c, $0), <($1, 100))]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))], variablesSet=[[$cor0]]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> ($1, $3))], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalAggregate(group=[{0}], EXPR$0=[SUM($1)]) +- LogicalProject(e1=[$0], e=[$0]) +- LogicalFilter(condition=[AND(<($1, 100), IS NOT NULL($0))]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -3998,14 +3998,14 @@ LogicalProject(b=[$1]) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), EXISTS({ LogicalFilter(condition=[AND(=($cor0.b, $1), >(CAST($2):BIGINT, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), >=(CAST($2):BIGINT, 1), EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -4015,14 +4015,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($0, $6)], joinType=[semi]) :- LogicalJoin(condition=[=($1, $6)], joinType=[semi]) : :- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, r]]) : +- LogicalProject(j=[$1]) : +- LogicalFilter(condition=[>(CAST($2):BIGINT, 50)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 100)]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -4035,14 +4035,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), EXISTS({ LogicalFilter(condition=[AND(=($cor0.b, $1), >(CAST($2):BIGINT, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), >=(CAST($2):BIGINT, 1), EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -4052,14 +4052,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalJoin(condition=[=($0, $6)], joinType=[semi]) :- LogicalJoin(condition=[=($1, $6)], joinType=[semi]) : :- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, r]]) : +- LogicalProject(j=[$1]) : +- LogicalFilter(condition=[>(CAST($2):BIGINT, 50)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 100)]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -4071,12 +4071,12 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) @@ -4084,14 +4084,14 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[=($0, $3)], joinType=[semi]) :- LogicalJoin(condition=[$3], joinType=[semi]) - : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : +- LogicalProject($f0=[IS NOT NULL($0)]) : +- LogicalAggregate(group=[{}], m=[MIN($0)]) : +- LogicalProject(i=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, r]]) +- LogicalProject(i=[$0]) +- LogicalFilter(condition=[<($1, 100)]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -4104,12 +4104,12 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(EXISTS({ LogicalFilter(condition=[<>($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), EXISTS({ LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -4117,15 +4117,15 @@ LogicalFilter(condition=[<($1, 100)]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[$3], joinType=[semi]) :- LogicalJoin(condition=[<>($0, $3)], joinType=[semi]) - : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : +- LogicalProject(d=[$0]) : +- LogicalFilter(condition=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, r]]) +- LogicalProject($f0=[IS NOT NULL($0)]) +- LogicalAggregate(group=[{}], m=[MIN($0)]) +- LogicalProject(i=[true]) +- LogicalFilter(condition=[<($1, 100)]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -4138,25 +4138,25 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[AND(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) +LogicalTableScan(table=[[default_catalog, default_database, t]]) }), =($cor0.a, $0))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -4170,26 +4170,26 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=($cor0.d, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -4202,14 +4202,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalFilter(condition=[AND(=($0, $2), EXISTS({ LogicalFilter(condition=[>($0, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }), >=($1, 1), EXISTS({ LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -4219,18 +4219,18 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3]) +- LogicalJoin(condition=[$4], joinType=[semi]) :- LogicalJoin(condition=[$4], joinType=[semi]) : :- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) : +- LogicalProject($f0=[IS NOT NULL($0)]) : +- LogicalAggregate(group=[{}], m=[MIN($0)]) : +- LogicalProject(i=[true]) : +- LogicalFilter(condition=[>($0, 50)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) +- LogicalProject($f0=[IS NOT NULL($0)]) +- LogicalAggregate(group=[{}], m=[MIN($0)]) +- LogicalProject(i=[true]) +- LogicalFilter(condition=[<($1, 100)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -4244,13 +4244,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -4258,13 +4258,13 @@ LogicalProject(j=[$1]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(=($1, $3), =($2, $4))], joinType=[semi]) :- LogicalJoin(condition=[AND(=($0, $3), =($2, $4))], joinType=[semi]) - : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : +- LogicalProject(d=[$0], f=[$2]) : +- LogicalFilter(condition=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, r]]) +- LogicalProject(j=[$1], k=[$2]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -4278,13 +4278,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) LogicalFilter(condition=[AND(=($cor0.a, $0), <>($2, _UTF-16LE'test'))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -4292,13 +4292,13 @@ LogicalProject(j=[$1]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(=($1, $3), =($0, $4))], joinType=[semi]) :- LogicalJoin(condition=[AND(=($0, $3), =($2, $4))], joinType=[semi]) - : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, l]]) : +- LogicalProject(d=[$0], f=[$2]) : +- LogicalFilter(condition=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, r]]) +- LogicalProject(j=[$1], i=[$0]) +- LogicalFilter(condition=[<>($2, _UTF-16LE'test')]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -4314,24 +4314,24 @@ LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), IN($2, { LogicalProject(k=[$2]) LogicalFilter(condition=[=($cor1.e, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor1]]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -4345,12 +4345,12 @@ LogicalProject(a=[$0]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(c=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), IN($1, { LogicalProject(f=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -4358,12 +4358,12 @@ LogicalProject(f=[$1]) LogicalProject(a=[$0]) +- LogicalJoin(condition=[=($1, $2)], joinType=[semi]) :- LogicalJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[semi]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(c=[$0], d=[$1]) : +- LogicalFilter(condition=[true]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(f=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -4376,13 +4376,13 @@ LogicalProject(a=[$0]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(c=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), IN($1, { LogicalProject(e=[$0]) LogicalFilter(condition=[=($1, $cor0.b)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -4390,13 +4390,13 @@ LogicalProject(e=[$0]) LogicalProject(a=[$0], b=[$1]) +- LogicalJoin(condition=[AND(=($1, $4), =($3, $1))], joinType=[semi]) :- LogicalJoin(condition=[=($0, $2)], joinType=[semi]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(c=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(e=[$0], f=[$1], e0=[CAST($0):BIGINT]) +- LogicalProject(e=[$0], f=[$1]) +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -4411,23 +4411,23 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(c=[$0]) LogicalFilter(condition=[AND(IN($1, { LogicalProject(f=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }), =($cor0.b, $1))]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -4443,24 +4443,24 @@ LogicalProject(c=[$0]) LogicalFilter(condition=[IN($1, { LogicalProject(f=[$1]) LogicalFilter(condition=[=($0, $cor0.c)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })], variablesSet=[[$cor0]]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -4473,12 +4473,12 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(c=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), IN($1, { LogicalProject(e=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -4486,12 +4486,12 @@ LogicalProject(e=[$0]) LogicalProject(a=[$0], b=[$1]) +- LogicalJoin(condition=[=($1, $3)], joinType=[semi]) :- LogicalJoin(condition=[=($0, $2)], joinType=[semi]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(c=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(e=[$0], e0=[CAST($0):BIGINT]) +- LogicalProject(e=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -4506,23 +4506,23 @@ LogicalProject(a=[$0], b=[$1]) LogicalProject(c=[$0]) LogicalFilter(condition=[IN($1, { LogicalProject(f=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) })]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -4536,13 +4536,13 @@ LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[AND(IN(+($0, 1), { LogicalProject(c=[$0]) LogicalFilter(condition=[>($1, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + LogicalTableScan(table=[[default_catalog, default_database, y]]) }), IN(*($1, 2), { LogicalProject(e=[$0]) LogicalFilter(condition=[<($1, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -4553,14 +4553,14 @@ LogicalProject(a=[$0], b=[$1]) :- LogicalProject(a=[$0], b=[$1], $f2=[$2], $f3=[*($1, 2)]) : +- LogicalJoin(condition=[=($2, $3)], joinType=[semi]) : :- LogicalProject(a=[$0], b=[$1], $f2=[+($0, 1)]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(c=[$0]) : +- LogicalFilter(condition=[>($1, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(e=[$0], e0=[CAST($0):BIGINT]) +- LogicalProject(e=[$0]) +- LogicalFilter(condition=[<($1, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubqueryCorrelateVariablesValidationTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubqueryCorrelateVariablesValidationTest.xml index 9c42ceef11f5d..b6bb678d8c739 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubqueryCorrelateVariablesValidationTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubqueryCorrelateVariablesValidationTest.xml @@ -30,10 +30,10 @@ LogicalProject(variablesSet=[[$cor0]], min_t1d=[$SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) LogicalProject($f0=[$cor0.t1d]) LogicalFilter(condition=[=($0, _UTF-16LE'test')]) - LogicalTableScan(table=[[default_catalog, default_database, t3, source: [TestTableSource(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t3]]) })]) +- LogicalFilter(condition=[=($0, _UTF-16LE'test')]) - +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) ]]> @@ -41,16 +41,16 @@ LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) LogicalProject(min_t1d=[$10]) +- LogicalJoin(condition=[=($3, $9)], joinType=[left]) :- LogicalFilter(condition=[=($0, _UTF-16LE'test')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) +- LogicalAggregate(group=[{0}], EXPR$0=[MIN($1)]) +- LogicalProject(t1d=[$9], $f0=[$9]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalFilter(condition=[=($0, _UTF-16LE'test')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t3, source: [TestTableSource(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t3]]) +- LogicalAggregate(group=[{0}]) +- LogicalProject(t1d=[$3]) +- LogicalFilter(condition=[=($0, _UTF-16LE'test')]) - +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) ]]> @@ -69,15 +69,15 @@ LogicalProject(variablesSet=[[$cor0]], min_t3d=[$SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) LogicalProject(t3d=[$3]) LogicalFilter(condition=[=($0, $cor0.t1a)]) - LogicalTableScan(table=[[default_catalog, default_database, t3, source: [TestTableSource(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t3]]) })], max_t2h=[$SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) LogicalProject(t2h=[$7]) LogicalFilter(condition=[=($0, $cor0.t1a)]) - LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t2]]) })]) +- LogicalFilter(condition=[=($0, _UTF-16LE'test')]) - +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) ]]> @@ -87,15 +87,15 @@ LogicalProject(min_t3d=[$9], max_t2h=[$11]) :- LogicalProject(t1a=[$0], t1b=[$1], t1c=[$2], t1d=[$3], t1e=[$4], t1f=[$5], t1g=[$6], t1h=[$7], t1i=[$8], EXPR$0=[$10]) : +- LogicalJoin(condition=[=($0, $9)], joinType=[left]) : :- LogicalFilter(condition=[=($0, _UTF-16LE'test')]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : +- LogicalAggregate(group=[{0}], EXPR$0=[MIN($1)]) : +- LogicalProject(t3a=[$0], t3d=[$3]) : +- LogicalFilter(condition=[IS NOT NULL($0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t3, source: [TestTableSource(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t3]]) +- LogicalAggregate(group=[{0}], EXPR$0=[MAX($1)]) +- LogicalProject(t2a=[$0], t2h=[$7]) +- LogicalFilter(condition=[IS NOT NULL($0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) ]]> @@ -116,16 +116,16 @@ LogicalProject(t1a=[$0]) LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) LogicalProject(t2h=[$7]) LogicalJoin(condition=[=($0, $cor0.t1a)], joinType=[left]) - LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i)]]]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t2]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.xml index bb54a34678972..983f5421b9475 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.xml @@ -25,7 +25,7 @@ limitations under the License. LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], EXPR$0=[AVG($2)]) +- LogicalProject(a=[$0], c=[$2], d=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -36,7 +36,7 @@ Calc(select=[EXPR$0]) +- LocalHashAggregate(groupBy=[a, c, $e], select=[a, c, $e, Partial_AVG(d) AS (sum$0, count$1)]) +- Expand(projects=[{a, c, d, 0 AS $e}, {a, null AS c, d, 1 AS $e}, {null AS a, c, d, 2 AS $e}, {null AS a, null AS c, d, 3 AS $e}]) +- Calc(select=[a, c, d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c, d]) ]]> @@ -49,7 +49,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], EXPR$0=[COUNT($2)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -60,7 +60,7 @@ Calc(select=[EXPR$0]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_COUNT(a) AS count$0]) +- Expand(projects=[{b, c, a, 0 AS $e}, {b, null AS c, a, 1 AS $e}, {null AS b, c, a, 2 AS $e}, {null AS b, null AS c, a, 3 AS $e}]) +- Calc(select=[b, c, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c, d]) ]]> @@ -73,7 +73,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1}, {0, 2}]], EXPR$0=[AVG($3)]) +- LogicalProject(b=[$1], c=[$2], d=[$3], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -82,7 +82,7 @@ Calc(select=[EXPR$0]) +- HashAggregate(isMerge=[false], groupBy=[b, c, d, $e], select=[b, c, d, $e, AVG(a) AS EXPR$0]) +- Exchange(distribution=[hash[b, c, d, $e]]) +- Expand(projects=[{a, b, c, null AS d, 1 AS $e}, {a, b, null AS c, d, 2 AS $e}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c, d]) ]]> @@ -95,7 +95,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1}, {0, 2}]], EXPR$0=[COUNT()]) +- LogicalProject(b=[$1], c=[$2], d=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -105,7 +105,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[hash[b, c, d, $e]]) +- Expand(projects=[{b, c, null AS d, 1 AS $e}, {b, null AS c, d, 2 AS $e}]) +- Calc(select=[b, c, d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c, d]) ]]> @@ -118,7 +118,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$0=[AVG($2)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -129,7 +129,7 @@ Calc(select=[EXPR$0]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)]) +- Expand(projects=[{b, c, a, 0 AS $e}, {b, null AS c, a, 1 AS $e}, {null AS b, null AS c, a, 3 AS $e}]) +- Calc(select=[b, c, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c, d]) ]]> @@ -142,7 +142,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$0=[COUNT()]) +- LogicalProject(b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -153,7 +153,7 @@ Calc(select=[EXPR$0]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_COUNT(*) AS count1$0]) +- Expand(projects=[{b, c, 0 AS $e}, {b, null AS c, 1 AS $e}, {null AS b, null AS c, 3 AS $e}]) +- Calc(select=[b, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c, d]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.xml index 3fb60f0a0932d..7dabfe77a3c2f 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.xml @@ -25,7 +25,7 @@ limitations under the License. LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], EXPR$0=[AVG($2)]) +- LogicalProject(a=[$0], c=[$2], d=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -38,7 +38,7 @@ Calc(select=[EXPR$0]) +- Sort(orderBy=[a ASC, c ASC, $e ASC]) +- Expand(projects=[{a, c, d, 0 AS $e}, {a, null AS c, d, 1 AS $e}, {null AS a, c, d, 2 AS $e}, {null AS a, null AS c, d, 3 AS $e}]) +- Calc(select=[a, c, d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c, d]) ]]> @@ -51,7 +51,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], EXPR$0=[COUNT($2)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -64,7 +64,7 @@ Calc(select=[EXPR$0]) +- Sort(orderBy=[b ASC, c ASC, $e ASC]) +- Expand(projects=[{b, c, a, 0 AS $e}, {b, null AS c, a, 1 AS $e}, {null AS b, c, a, 2 AS $e}, {null AS b, null AS c, a, 3 AS $e}]) +- Calc(select=[b, c, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c, d]) ]]> @@ -77,7 +77,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], EXPR$0=[weightedAvg($2, $3)]) +- LogicalProject(a=[$0], b=[$1], d=[$3], $f3=[1]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -88,7 +88,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[hash[a, b, $e]]) +- Expand(projects=[{a, b, d, $f3, 0 AS $e}, {a, null AS b, d, $f3, 1 AS $e}, {null AS a, b, d, $f3, 2 AS $e}, {null AS a, null AS b, d, $f3, 3 AS $e}]) +- Calc(select=[a, b, d, 1 AS $f3]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c, d]) ]]> @@ -101,7 +101,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1}, {0, 2}]], EXPR$0=[AVG($3)]) +- LogicalProject(b=[$1], c=[$2], d=[$3], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -111,7 +111,7 @@ Calc(select=[EXPR$0]) +- Sort(orderBy=[b ASC, c ASC, d ASC, $e ASC]) +- Exchange(distribution=[hash[b, c, d, $e]]) +- Expand(projects=[{a, b, c, null AS d, 1 AS $e}, {a, b, null AS c, d, 2 AS $e}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c, d]) ]]> @@ -124,7 +124,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1}, {0, 2}]], EXPR$0=[COUNT()]) +- LogicalProject(b=[$1], c=[$2], d=[$3]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -135,7 +135,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[hash[b, c, d, $e]]) +- Expand(projects=[{b, c, null AS d, 1 AS $e}, {b, null AS c, d, 2 AS $e}]) +- Calc(select=[b, c, d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c, d]) ]]> @@ -148,7 +148,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$0=[weightedAvg($2, $3)]) +- LogicalProject(b=[$1], c=[$2], a=[$0], $f3=[1]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -159,7 +159,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[hash[b, c, $e]]) +- Expand(projects=[{b, c, a, $f3, 0 AS $e}, {b, null AS c, a, $f3, 1 AS $e}, {null AS b, null AS c, a, $f3, 3 AS $e}]) +- Calc(select=[b, c, a, 1 AS $f3]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c, d]) ]]> @@ -172,7 +172,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1}, {0, 2}]], EXPR$0=[weightedAvg($3, $4)]) +- LogicalProject(b=[$1], c=[$2], d=[$3], a=[$0], $f4=[1]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -183,7 +183,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[hash[b, c, d, $e]]) +- Expand(projects=[{b, c, null AS d, a, $f4, 1 AS $e}, {b, null AS c, d, a, $f4, 2 AS $e}]) +- Calc(select=[b, c, d, a, 1 AS $f4]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c, d]) ]]> @@ -196,7 +196,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$0=[AVG($2)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -209,7 +209,7 @@ Calc(select=[EXPR$0]) +- Sort(orderBy=[b ASC, c ASC, $e ASC]) +- Expand(projects=[{b, c, a, 0 AS $e}, {b, null AS c, a, 1 AS $e}, {null AS b, null AS c, a, 3 AS $e}]) +- Calc(select=[b, c, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c, d]) ]]> @@ -222,7 +222,7 @@ Calc(select=[EXPR$0]) LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$0=[COUNT()]) +- LogicalProject(b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, t]]) ]]> @@ -235,7 +235,7 @@ Calc(select=[EXPR$0]) +- Sort(orderBy=[b ASC, c ASC, $e ASC]) +- Expand(projects=[{b, c, 0 AS $e}, {b, null AS c, 1 AS $e}, {null AS b, null AS c, 3 AS $e}]) +- Calc(select=[b, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c, d]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.xml index 0d00419daaf81..ea6fc21c5694d 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.xml @@ -26,7 +26,7 @@ FROM z GROUP BY d]]> ($0, 5))], b=[$1], $f6=[IS TRUE(>($1, 3))]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -40,7 +40,7 @@ HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_MIN(min$0) AS EXPR$1 +- LocalHashAggregate(groupBy=[d, c, $f4, b, $f6, $e], select=[d, c, $f4, b, $f6, $e, Partial_MAX(e) AS max$0, Partial_MAX(e) FILTER $f2 AS max$1]) +- Expand(projects=[{d, e, $f2, c, $f4, null AS b, null AS $f6, 3 AS $e}, {d, e, $f2, c, null AS $f4, null AS b, null AS $f6, 7 AS $e}, {d, e, $f2, null AS c, null AS $f4, b, $f6, 12 AS $e}, {d, e, $f2, null AS c, null AS $f4, null AS b, null AS $f6, 15 AS $e}]) +- Calc(select=[d, e, IS TRUE(<(a, 10)) AS $f2, c, IS TRUE(>(a, 5)) AS $f4, b, IS TRUE(>(b, 3)) AS $f6]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, z, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, z]], fields=[a, b, c, d, e]) ]]> @@ -59,8 +59,8 @@ LogicalProject(EXPR$0=[$1]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), LIKE($2, _UTF-16LE'He%'))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -71,10 +71,10 @@ Calc(select=[EXPR$0]) +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, d], build=[left]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b], where=[LIKE(c, 'He%')]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) ]]> @@ -95,7 +95,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rk=[$3]) +- LogicalFilter(condition=[<=($3, 10)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rk=[RANK() OVER (PARTITION BY $0, $2 ORDER BY $1 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -109,7 +109,7 @@ HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1 +- Exchange(distribution=[hash[a, c]]) +- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=10], partitionBy=[a, c], orderBy=[b ASC], global=[false], select=[a, b, c]) +- Sort(orderBy=[a ASC, c ASC, b ASC]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.xml index e0bed9429d657..cd844314603ec 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.xml @@ -31,7 +31,7 @@ LogicalProject(a=[$0]) +- LogicalProject(a=[$0], rk=[RANK() OVER (PARTITION BY $0 ORDER BY $1 NULLS FIRST)]) +- LogicalAggregate(group=[{0}], agg#0=[SUM($1)]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -43,7 +43,7 @@ Calc(select=[a]) +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) ]]> @@ -62,7 +62,7 @@ LogicalProject(a=[$0]) +- LogicalProject(a=[$0], rk=[RANK() OVER (PARTITION BY $0 ORDER BY $1 NULLS FIRST)]) +- LogicalAggregate(group=[{0}], agg#0=[SUM($1)]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -74,7 +74,7 @@ Calc(select=[a]) +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) ]]> @@ -95,7 +95,7 @@ LogicalProject(a=[$0], b=[$1], rk=[$2], rk1=[$3]) +- LogicalProject(a=[$0], b=[$1], rk=[$2], rk1=[RANK() OVER (PARTITION BY $0 ORDER BY $1 NULLS FIRST)]) +- LogicalFilter(condition=[<=($2, 5)]) +- LogicalProject(a=[$0], b=[$1], rk=[RANK() OVER (PARTITION BY $0 ORDER BY $1 NULLS FIRST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -107,7 +107,7 @@ Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=5], partitionBy=[a], order +- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=5], partitionBy=[a], orderBy=[b ASC], global=[false], select=[a, b]) +- Sort(orderBy=[a ASC, b ASC]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.xml index ae00d6a0c9f55..fb75190d2c316 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.xml @@ -31,8 +31,8 @@ LogicalProject(EXPR$0=[$1]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), LIKE($2, _UTF-16LE'He%'))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -43,10 +43,10 @@ Calc(select=[EXPR$0]) +- SortMergeJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, d]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b], where=[LIKE(c, 'He%')]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) ]]> @@ -60,7 +60,7 @@ FROM z GROUP BY d]]> ($0, 5))], b=[$1], $f6=[IS TRUE(>($1, 3))]) - +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -76,7 +76,7 @@ SortAggregate(isMerge=[true], groupBy=[d], select=[d, Final_MIN(min$0) AS EXPR$1 +- Exchange(distribution=[hash[d, c, $f4, b, $f6, $e]]) +- Expand(projects=[{d, e, $f2, c, $f4, null AS b, null AS $f6, 3 AS $e}, {d, e, $f2, c, null AS $f4, null AS b, null AS $f6, 7 AS $e}, {d, e, $f2, null AS c, null AS $f4, b, $f6, 12 AS $e}, {d, e, $f2, null AS c, null AS $f4, null AS b, null AS $f6, 15 AS $e}]) +- Calc(select=[d, e, IS TRUE(<(a, 10)) AS $f2, c, IS TRUE(>(a, 5)) AS $f4, b, IS TRUE(>(b, 3)) AS $f6]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, z, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- TableSourceScan(table=[[default_catalog, default_database, z]], fields=[a, b, c, d, e]) ]]> @@ -95,8 +95,8 @@ LogicalProject(EXPR$0=[$1]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), LIKE($2, _UTF-16LE'He%'))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -108,10 +108,10 @@ Calc(select=[EXPR$0]) +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, d], build=[left]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b], where=[LIKE(c, 'He%')]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) ]]> 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 20aac897cd2af..4a590a1f777b6 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 @@ -23,13 +23,13 @@ limitations under the License. @@ -51,7 +51,7 @@ LogicalProject(a=[$0], r=[$1]) +- LogicalProject(a=[$0], r=[random_udf($1)]) +- LogicalProject(a=[$0], b=[$1], c1=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableFunctionScan(invocation=[str_split($cor0.c)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) ]]> @@ -60,7 +60,7 @@ LogicalProject(a=[$0], r=[$1]) Calc(select=[a, r], where=[>(r, 10)]) +- Calc(select=[a, random_udf(b) AS r]) +- Correlate(invocation=[str_split($cor0.c)], correlate=[table(str_split($cor0.c))], select=[a,b,c,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, VARCHAR(2147483647) EXPR$0)], joinType=[INNER]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -73,14 +73,14 @@ Calc(select=[a, r], where=[>(r, 10)]) LogicalProject(a=[$0], a1=[$1]) +- LogicalFilter(condition=[>($1, 10)]) +- LogicalProject(a=[$0], a1=[random_udf($1)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 10)]) +- Calc(select=[a, random_udf(b) AS a1]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -93,13 +93,13 @@ Calc(select=[a, a1], where=[(a1 > 10)]) LogicalProject(a=[$0]) +- LogicalFilter(condition=[>(random_udf($1), 10)]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> (random_udf(b), 10)]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -111,13 +111,13 @@ Calc(select=[a], where=[>(random_udf(b), 10)]) ($1, 20))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 20))]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -163,13 +163,13 @@ Calc(select=[MAP('a', 0.12, 'b', 0.5) AS EXPR$0]) ($0, 20))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -181,13 +181,13 @@ Calc(select=[a, b, c], where=[SEARCH(a, Sarg[(-∞..10), (20..+∞)])]) @@ -202,13 +202,13 @@ WHERE b = DATE '1984-07-12' AND c = TIME '14:34:24' AND d = TIMESTAMP '1984-07-1 @@ -220,13 +220,13 @@ Calc(select=[a, CAST(1984-07-12 AS DATE) AS b, CAST(14:34:24 AS TIME(0)) AS c, C @@ -238,13 +238,13 @@ Calc(select=[a, b, CAST('xx' AS VARCHAR(2147483647)) AS c], where=[(SEARCH(b, Sa @@ -258,13 +258,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[<($0, 50)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>($1, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 0) AND (a < 50))]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -275,13 +275,13 @@ Calc(select=[a, b, c], where=[((b > 0) AND (a < 50))]) @@ -293,13 +293,13 @@ Calc(select=[a._1 AS _1, a._2 AS _2, c, b._1 AS _10, b._2 AS _20]) @@ -310,13 +310,13 @@ Calc(select=[c]) @@ -328,13 +328,13 @@ Calc(select=[MAP(a, c) AS EXPR$0]) @@ -346,13 +346,13 @@ Calc(select=[a, b, c], where=[SEARCH(a, Sarg[1, 10; NULL AS TRUE])]) @@ -364,13 +364,13 @@ Calc(select=[a, b, c], where=[(SEARCH(b, Sarg[(-∞..1), (1..3), (3..4), (4..5), ($1, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 0)]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -381,13 +381,13 @@ Calc(select=[a, b, c], where=[(b > 0)]) @@ -399,13 +399,13 @@ Calc(select=[a, c]) ($1, 2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 2)]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -416,13 +416,13 @@ Calc(select=[a, (b + 1) AS EXPR$1], where=[(b > 2)]) @@ -433,12 +433,12 @@ Calc(select=[IF(SEARCH(c, Sarg[_UTF-16LE''; NULL AS TRUE]), 'a', 'b') AS EXPR$0] @@ -449,13 +449,13 @@ LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable4, sourc @@ -472,13 +472,13 @@ FROM MyTable @@ -490,12 +490,12 @@ Calc(select=[a, b, c, 1984-07-12 AS EXPR$3, 14:34:24 AS EXPR$4, 1984-07-12 14:34 @@ -525,13 +525,13 @@ Calc(select=[(CAST(my_row AS RecordType(INTEGER a, VARCHAR(2147483647) b)) = CAS diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.xml index 616eed09d5623..7570f11349e1b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.xml @@ -22,7 +22,7 @@ limitations under the License. LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a, b, c]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[b, cnt]) +- LogicalProject(b=[$0], cnt=[$1]) @@ -33,11 +33,11 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fie :- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[b, cnt]) +- LogicalProject(b=[$0], cnt=[$1]) @@ -48,18 +48,18 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fiel :- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -94,7 +94,7 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink1`], fi +- LogicalProject(sum_a=[$1], c=[$0]) +- LogicalAggregate(group=[{0}], sum_a=[SUM($1)]) +- LogicalProject(c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[total_min]) +- LogicalAggregate(group=[{}], total_min=[MIN($0)]) @@ -102,7 +102,7 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink2`], fi +- LogicalProject(sum_a=[$1], c=[$0]) +- LogicalAggregate(group=[{0}], sum_a=[SUM($1)]) +- LogicalProject(c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -114,7 +114,7 @@ LegacySink(name=[`default_catalog`.`default_database`.`retractSink1`], fields=[t +- GroupAggregate(groupBy=[c], select=[c, SUM(a) AS sum_a], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[c]], changelogMode=[I]) +- Calc(select=[c, a], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) LegacySink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[total_min], changelogMode=[NONE]) +- GroupAggregate(select=[MIN_RETRACT(sum_a) AS total_min], changelogMode=[I,UB,UA,D]) @@ -123,7 +123,7 @@ LegacySink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[t +- GroupAggregate(groupBy=[c], select=[c, SUM(a) AS sum_a], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[c]], changelogMode=[I]) +- Calc(select=[c, a], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -135,30 +135,30 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink1`], fie :- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b1=[$1]) +- LogicalProject(a=[$2], b1=[$1]) +- LogicalFilter(condition=[=($2, $0)]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a, b1]) +- LogicalUnion(all=[true]) :- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b1=[$1]) +- LogicalProject(a=[$2], b1=[$1]) +- LogicalFilter(condition=[=($2, $0)]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -166,28 +166,28 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink2`], fie LegacySink(name=[`default_catalog`.`default_database`.`appendSink1`], fields=[a, b1], changelogMode=[NONE]) +- Union(all=[true], union=[a, b], changelogMode=[I]) :- Calc(select=[a, b], where=[<=(a, 10)], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Calc(select=[a0 AS a, b AS b1], changelogMode=[I]) +- Join(joinType=[InnerJoin], where=[=(a0, a)], select=[a, b, a0], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], changelogMode=[I]) :- Exchange(distribution=[hash[a]], changelogMode=[I]) : +- Calc(select=[a, b], where=[<=(a, 10)], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Exchange(distribution=[hash[a]], changelogMode=[I]) +- Calc(select=[a], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e], changelogMode=[I]) LegacySink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a, b1], changelogMode=[NONE]) +- Union(all=[true], union=[a, b], changelogMode=[I]) :- Calc(select=[a, b], where=[<=(a, 10)], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Calc(select=[a0 AS a, b AS b1], changelogMode=[I]) +- Join(joinType=[InnerJoin], where=[=(a0, a)], select=[a, b, a0], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], changelogMode=[I]) :- Exchange(distribution=[hash[a]], changelogMode=[I]) : +- Calc(select=[a, b], where=[<=(a, 10)], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Exchange(distribution=[hash[a]], changelogMode=[I]) +- Calc(select=[a], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e], changelogMode=[I]) ]]> @@ -200,23 +200,23 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink1`], fie +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a, b1]) +- LogicalUnion(all=[true]) :- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b1=[$1]) +- LogicalProject(a=[$2], b1=[$1]) +- LogicalFilter(condition=[=($2, $0)]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -226,23 +226,23 @@ LegacySink(name=[`default_catalog`.`default_database`.`appendSink1`], fields=[a, +- Join(joinType=[InnerJoin], where=[=(a0, a)], select=[a, b, a0], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], changelogMode=[I]) :- Exchange(distribution=[hash[a]], changelogMode=[I]) : +- Calc(select=[a, b], where=[<=(a, 10)], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Exchange(distribution=[hash[a]], changelogMode=[I]) +- Calc(select=[a], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e], changelogMode=[I]) LegacySink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a, b1], changelogMode=[NONE]) +- Union(all=[true], union=[a, b], changelogMode=[I]) :- Calc(select=[a, b], where=[<=(a, 10)], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Calc(select=[a0 AS a, b AS b1], changelogMode=[I]) +- Join(joinType=[InnerJoin], where=[=(a0, a)], select=[a, b, a0], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], changelogMode=[I]) :- Exchange(distribution=[hash[a]], changelogMode=[I]) : +- Calc(select=[a, b], where=[<=(a, 10)], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Exchange(distribution=[hash[a]], changelogMode=[I]) +- Calc(select=[a], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e], changelogMode=[I]) ]]> @@ -255,12 +255,12 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink1`], fie +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a1=[$0], b=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a2=[$0], c=[$2]) +- LogicalFilter(condition=[>=($1, 5)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>=($0, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a1, b, c1]) +- LogicalProject(a1=[$2], b=[$3], c1=[$1]) @@ -270,18 +270,18 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink2`], fie : +- LogicalFilter(condition=[<($1, 5)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalFilter(condition=[>=($0, 0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a1=[$0], b=[$1], c2=[$3]) +- LogicalFilter(condition=[=($0, $2)]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a1=[$0], b=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a2=[$0], c=[$2]) +- LogicalFilter(condition=[>=($1, 5)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>=($0, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -291,11 +291,11 @@ LegacySink(name=[`default_catalog`.`default_database`.`appendSink1`], fields=[a1 +- Join(joinType=[InnerJoin], where=[=(a, a2)], select=[a, b, a2, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], changelogMode=[I]) :- Exchange(distribution=[hash[a]], changelogMode=[I]) : +- Calc(select=[a, b], where=[<=(a, 10)], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Exchange(distribution=[hash[a2]], changelogMode=[I]) +- Calc(select=[a AS a2, c], where=[>=(b, 5)], changelogMode=[I]) +- Calc(select=[a, b, c], where=[>=(a, 0)], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) LegacySink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a1, b, c1], changelogMode=[NONE]) +- Calc(select=[a1, b, c1], changelogMode=[I]) @@ -303,18 +303,18 @@ LegacySink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a1 :- Exchange(distribution=[hash[a3]], changelogMode=[I]) : +- Calc(select=[a AS a3, c AS c1], where=[<(b, 5)], changelogMode=[I]) : +- Calc(select=[a, b, c], where=[>=(a, 0)], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Exchange(distribution=[hash[a1]], changelogMode=[I]) +- Calc(select=[a1, b], changelogMode=[I]) +- Calc(select=[a AS a1, b, c AS c2], changelogMode=[I]) +- Join(joinType=[InnerJoin], where=[=(a, a2)], select=[a, b, a2, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], changelogMode=[I]) :- Exchange(distribution=[hash[a]], changelogMode=[I]) : +- Calc(select=[a, b], where=[<=(a, 10)], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Exchange(distribution=[hash[a2]], changelogMode=[I]) +- Calc(select=[a AS a2, c], where=[>=(b, 5)], changelogMode=[I]) +- Calc(select=[a, b, c], where=[>=(a, 0)], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -325,13 +325,13 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink1`], fi +- LogicalAggregate(group=[{}], total_sum=[SUM($0)]) +- LogicalProject(a=[$0]) +- LogicalProject(a=[random_udf($0)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[total_min]) +- LogicalAggregate(group=[{}], total_min=[MIN($0)]) +- LogicalProject(a=[$0]) +- LogicalProject(a=[random_udf($0)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -340,13 +340,13 @@ LegacySink(name=[`default_catalog`.`default_database`.`retractSink1`], fields=[t +- GroupAggregate(select=[SUM(a) AS total_sum], changelogMode=[I,UB,UA]) +- Exchange(distribution=[single], changelogMode=[I]) +- Calc(select=[random_udf(a) AS a], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) LegacySink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[total_min], changelogMode=[NONE]) +- GroupAggregate(select=[MIN(a) AS total_min], changelogMode=[I,UB,UA]) +- Exchange(distribution=[single], changelogMode=[I]) +- Calc(select=[random_udf(a) AS a], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -364,12 +364,12 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink1`], fi : +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) : :- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2]) : : +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalTableFunctionScan(invocation=[split($cor0.c)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) +- LogicalProject(a=[$0], total_c=[0]) +- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2]) +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[a, total_c]) +- LogicalProject(a=[$0], total_c=[$1]) @@ -382,12 +382,12 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink2`], fi : +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) : :- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2]) : : +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalTableFunctionScan(invocation=[split($cor0.c)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) +- LogicalProject(a=[$0], total_c=[0]) +- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2]) +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -401,10 +401,10 @@ LegacySink(name=[`default_catalog`.`default_database`.`retractSink1`], fields=[a : +- Calc(select=[a, b, EXPR$0 AS c], changelogMode=[I]) : +- Correlate(invocation=[split($cor0.c)], correlate=[table(split($cor0.c))], select=[a,b,c,EXPR$0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) EXPR$0)], joinType=[INNER], changelogMode=[I]) : +- Calc(select=[a, -(b, MOD(b, 300)) AS b, c], where=[AND(>=(b, UNIX_TIMESTAMP('${startTime}')), <>(c, ''))], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Calc(select=[a, CAST(total_c AS BIGINT) AS total_c], where=[>(a, 50)], changelogMode=[I]) +- Calc(select=[a, 0 AS total_c], where=[>=(b, UNIX_TIMESTAMP('${startTime}'))], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) LegacySink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[a, total_c], changelogMode=[NONE]) +- Union(all=[true], union=[a, total_c], changelogMode=[I,UB,UA]) @@ -415,10 +415,10 @@ LegacySink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[a : +- Calc(select=[a, b, EXPR$0 AS c], changelogMode=[I]) : +- Correlate(invocation=[split($cor0.c)], correlate=[table(split($cor0.c))], select=[a,b,c,EXPR$0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) EXPR$0)], joinType=[INNER], changelogMode=[I]) : +- Calc(select=[a, -(b, MOD(b, 300)) AS b, c], where=[AND(>=(b, UNIX_TIMESTAMP('${startTime}')), <>(c, ''))], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Calc(select=[a, CAST(total_c AS BIGINT) AS total_c], where=[<(a, 50)], changelogMode=[I]) +- Calc(select=[a, 0 AS total_c], where=[>=(b, UNIX_TIMESTAMP('${startTime}'))], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -430,18 +430,18 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fiel +- LogicalProject(a=[$0]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(d=[$0], f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[total_min]) +- LogicalAggregate(group=[{}], total_min=[MIN($0)]) +- LogicalProject(a=[$0]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(d=[$0], f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -452,10 +452,10 @@ LegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[tot +- Union(all=[true], union=[a], changelogMode=[I]) :- Calc(select=[a], changelogMode=[I]) : +- Calc(select=[a, c], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Calc(select=[d AS a], changelogMode=[I]) +- Calc(select=[d, f], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[d, e, f], changelogMode=[I]) LegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[total_min], changelogMode=[NONE]) +- GroupAggregate(select=[MIN(a) AS total_min], changelogMode=[I,UB,UA]) @@ -463,10 +463,10 @@ LegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[to +- Union(all=[true], union=[a], changelogMode=[I]) :- Calc(select=[a], changelogMode=[I]) : +- Calc(select=[a, c], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Calc(select=[d AS a], changelogMode=[I]) +- Calc(select=[d, f], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[d, e, f], changelogMode=[I]) ]]> @@ -479,11 +479,11 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink1`], fi +- LogicalUnion(all=[true]) :- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], c=[$2]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalProject(d=[$0], f=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(a=[$0], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[total_min]) +- LogicalAggregate(group=[{}], total_min=[MIN($0)]) @@ -491,19 +491,19 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink2`], fi +- LogicalUnion(all=[true]) :- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], c=[$2]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalProject(d=[$0], f=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(a=[$0], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink3`], fields=[a]) +- LogicalProject(a=[$0]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(d=[$0], f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -515,12 +515,12 @@ LegacySink(name=[`default_catalog`.`default_database`.`retractSink1`], fields=[t :- Union(all=[true], union=[a], changelogMode=[I]) : :- Calc(select=[a], changelogMode=[I]) : : +- Calc(select=[a, c], changelogMode=[I]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) : +- Calc(select=[d AS a], changelogMode=[I]) : +- Calc(select=[d, f], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[d, e, f], changelogMode=[I]) +- Calc(select=[a], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c], changelogMode=[I]) LegacySink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[total_min], changelogMode=[NONE]) +- GroupAggregate(select=[MIN(a) AS total_min], changelogMode=[I,UB,UA]) @@ -529,21 +529,21 @@ LegacySink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[t :- Union(all=[true], union=[a], changelogMode=[I]) : :- Calc(select=[a], changelogMode=[I]) : : +- Calc(select=[a, c], changelogMode=[I]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) : +- Calc(select=[d AS a], changelogMode=[I]) : +- Calc(select=[d, f], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[d, e, f], changelogMode=[I]) +- Calc(select=[a], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c], changelogMode=[I]) LegacySink(name=[`default_catalog`.`default_database`.`appendSink3`], fields=[a], changelogMode=[NONE]) +- Union(all=[true], union=[a], changelogMode=[I]) :- Calc(select=[a], changelogMode=[I]) : +- Calc(select=[a, c], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Calc(select=[d AS a], changelogMode=[I]) +- Calc(select=[d, f], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[d, e, f], changelogMode=[I]) ]]> @@ -553,9 +553,9 @@ LegacySink(name=[`default_catalog`.`default_database`.`appendSink3`], fields=[a] LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a, c]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(d=[$0], f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[total_sum]) +- LogicalAggregate(group=[{}], total_sum=[SUM($0)]) @@ -564,11 +564,11 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fie :- LogicalProject(a=[$0], c=[$1]) : +- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], c=[$2]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalProject(d=[$0], f=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(a=[$0], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[total_min]) +- LogicalAggregate(group=[{}], total_min=[MIN($0)]) @@ -577,11 +577,11 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fiel :- LogicalProject(a=[$0], c=[$1]) : +- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], c=[$2]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalProject(d=[$0], f=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(a=[$0], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -589,9 +589,9 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fiel LegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a, c], changelogMode=[NONE]) +- Union(all=[true], union=[a, c], changelogMode=[I]) :- Calc(select=[a, c], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Calc(select=[d, f], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[d, e, f], changelogMode=[I]) LegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[total_sum], changelogMode=[NONE]) +- GroupAggregate(select=[SUM(a) AS total_sum], changelogMode=[I,UB,UA]) @@ -600,12 +600,12 @@ LegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[to :- Calc(select=[a], changelogMode=[I]) : +- Union(all=[true], union=[a, c], changelogMode=[I]) : :- Calc(select=[a, c], changelogMode=[I]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) : +- Calc(select=[d, f], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[d, e, f], changelogMode=[I]) +- Calc(select=[a], changelogMode=[I]) +- Calc(select=[a, c], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c], changelogMode=[I]) LegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[total_min], changelogMode=[NONE]) +- GroupAggregate(select=[MIN(a) AS total_min], changelogMode=[I,UA]) @@ -614,12 +614,12 @@ LegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[tot :- Calc(select=[a], changelogMode=[I]) : +- Union(all=[true], union=[a, c], changelogMode=[I]) : :- Calc(select=[a, c], changelogMode=[I]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) : +- Calc(select=[d, f], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[d, e, f], changelogMode=[I]) +- Calc(select=[a], changelogMode=[I]) +- Calc(select=[a, c], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -632,11 +632,11 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fiel +- LogicalUnion(all=[true]) :- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], c=[$2]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalProject(d=[$0], f=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(a=[$0], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[total_min]) +- LogicalAggregate(group=[{}], total_min=[MIN($0)]) @@ -644,11 +644,11 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fie +- LogicalUnion(all=[true]) :- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], c=[$2]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalProject(d=[$0], f=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(a=[$0], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -660,13 +660,13 @@ LegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[tot :- Union(all=[true], union=[a], changelogMode=[I]) : :- Calc(select=[a], changelogMode=[I]) : : +- Calc(select=[a, c], changelogMode=[I]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) : +- Calc(select=[d AS a], changelogMode=[I]) : +- Calc(select=[d, f], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[d, e, f], changelogMode=[I]) +- Calc(select=[a], changelogMode=[I]) +- Calc(select=[a, c], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c], changelogMode=[I]) LegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[total_min], changelogMode=[NONE]) +- GroupAggregate(select=[MIN(a) AS total_min], changelogMode=[I,UB,UA]) @@ -675,13 +675,13 @@ LegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[to :- Union(all=[true], union=[a], changelogMode=[I]) : :- Calc(select=[a], changelogMode=[I]) : : +- Calc(select=[a, c], changelogMode=[I]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) : +- Calc(select=[d AS a], changelogMode=[I]) : +- Calc(select=[d, f], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[d, e, f], changelogMode=[I]) +- Calc(select=[a], changelogMode=[I]) +- Calc(select=[a, c], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -691,7 +691,7 @@ LegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[to LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a, b, c]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink1`], fields=[a, b, c]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) @@ -700,11 +700,11 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink1`], fi :- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[b, cnt]) +- LogicalProject(b=[$0], cnt=[$1]) @@ -715,11 +715,11 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink2`], fi :- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[b, cnt]) +- LogicalProject(b=[$0], cnt=[$1]) @@ -730,27 +730,27 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fiel :- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> =(b, 5)], changelogMode=[I]) : +- Calc(select=[a, b, c], where=[LIKE(c, '%hello%')], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Calc(select=[a, b, c], where=[>=(b, 5)], changelogMode=[I]) +- Calc(select=[a, b, c], where=[LIKE(c, '%world%')], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) LegacySink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[b, cnt], changelogMode=[NONE]) +- GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS cnt], changelogMode=[I,UB,UA]) @@ -758,10 +758,10 @@ LegacySink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[b +- Union(all=[true], union=[b, a], changelogMode=[I]) :- Calc(select=[b, a], where=[<(b, 4)], changelogMode=[I]) : +- Calc(select=[a, b, c], where=[LIKE(c, '%hello%')], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Calc(select=[b, a], where=[<(b, 4)], changelogMode=[I]) +- Calc(select=[a, b, c], where=[LIKE(c, '%world%')], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) LegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[b, cnt], changelogMode=[NONE]) +- GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS cnt], changelogMode=[I,UB,UA]) @@ -769,10 +769,10 @@ LegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[b, +- Union(all=[true], union=[b, a], changelogMode=[I]) :- Calc(select=[b, a], where=[<(b, 4)], changelogMode=[I]) : +- Calc(select=[a, b, c], where=[LIKE(c, '%hello%')], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Calc(select=[b, a], where=[<(b, 4)], changelogMode=[I]) +- Calc(select=[a, b, c], where=[LIKE(c, '%world%')], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -782,7 +782,7 @@ LegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[b, LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[c, cnt]) +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)]) +- LogicalProject(c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -791,7 +791,7 @@ LegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[c, +- GroupAggregate(groupBy=[c], select=[c, COUNT(a) AS cnt], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[c]], changelogMode=[I]) +- Calc(select=[c, a], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -806,18 +806,18 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fiel : +- LogicalFilter(condition=[<($1, 5)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalFilter(condition=[>=($0, 0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a1=[$0], b=[$1], c2=[$3]) +- LogicalFilter(condition=[=($0, $2)]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a1=[$0], b=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a2=[$0], c=[$2]) +- LogicalFilter(condition=[>=($1, 5)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>=($0, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -827,16 +827,16 @@ LegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a1, +- Join(joinType=[InnerJoin], where=[=(a1, a3)], select=[a3, c1, a1, b], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], changelogMode=[I]) :- Exchange(distribution=[hash[a3]], changelogMode=[I]) : +- Calc(select=[a AS a3, c AS c1], where=[AND(>=(a, 0), <(b, 5))], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Exchange(distribution=[hash[a1]], changelogMode=[I]) +- Calc(select=[a AS a1, b], changelogMode=[I]) +- Join(joinType=[InnerJoin], where=[=(a, a2)], select=[a, b, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], changelogMode=[I]) :- Exchange(distribution=[hash[a]], changelogMode=[I]) : +- Calc(select=[a, b], where=[<=(a, 10)], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Exchange(distribution=[hash[a2]], changelogMode=[I]) +- Calc(select=[a AS a2], where=[AND(>=(a, 0), >=(b, 5))], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -848,14 +848,14 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fiel :- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b1=[$1]) +- LogicalProject(a=[$2], b1=[$1]) +- LogicalFilter(condition=[=($2, $0)]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a1=[$0], b1=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -864,12 +864,12 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fiel LegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a1, b1], changelogMode=[NONE]) +- Union(all=[true], union=[a, b], changelogMode=[I]) :- Calc(select=[a, b], where=[<=(a, 10)], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Calc(select=[a0 AS a, b AS b1], changelogMode=[I]) +- Join(joinType=[InnerJoin], where=[=(a0, a)], select=[a, b, a0], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], changelogMode=[I]) :- Exchange(distribution=[hash[a]], changelogMode=[I]) : +- Calc(select=[a, b], where=[<=(a, 10)], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Exchange(distribution=[hash[a]], changelogMode=[I]) +- Calc(select=[a], changelogMode=[I]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, d, e], changelogMode=[I]) @@ -885,7 +885,7 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fiel +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a1=[$0], b=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a3=[$0], b1=[$3], c1=[$1]) +- LogicalFilter(condition=[=($2, $0)]) +- LogicalJoin(condition=[true], joinType=[inner]) @@ -893,18 +893,18 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fiel : +- LogicalFilter(condition=[<($1, 5)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalFilter(condition=[>=($0, 0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a1=[$0], b=[$1], c2=[$3]) +- LogicalFilter(condition=[=($0, $2)]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a1=[$0], b=[$1]) : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a2=[$0], c=[$2]) +- LogicalFilter(condition=[>=($1, 5)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[>=($0, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -914,22 +914,22 @@ LegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a, +- Join(joinType=[InnerJoin], where=[=(a, a3)], select=[a, a3, b1, c1], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], changelogMode=[I]) :- Exchange(distribution=[hash[a]], changelogMode=[I]) : +- Calc(select=[a], where=[<=(a, 10)], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Exchange(distribution=[hash[a3]], changelogMode=[I]) +- Calc(select=[a3, b AS b1, c1], changelogMode=[I]) +- Join(joinType=[InnerJoin], where=[=(a1, a3)], select=[a3, c1, a1, b], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], changelogMode=[I]) :- Exchange(distribution=[hash[a3]], changelogMode=[I]) : +- Calc(select=[a AS a3, c AS c1], where=[AND(>=(a, 0), <(b, 5))], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Exchange(distribution=[hash[a1]], changelogMode=[I]) +- Calc(select=[a AS a1, b], changelogMode=[I]) +- Join(joinType=[InnerJoin], where=[=(a, a2)], select=[a, b, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], changelogMode=[I]) :- Exchange(distribution=[hash[a]], changelogMode=[I]) : +- Calc(select=[a, b], where=[<=(a, 10)], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Exchange(distribution=[hash[a2]], changelogMode=[I]) +- Calc(select=[a AS a2], where=[AND(>=(a, 0), >=(b, 5))], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -943,9 +943,9 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fiel : +- LogicalFilter(condition=[AND(=($1, $4), =($0, $6))]) : +- LogicalJoin(condition=[true], joinType=[inner]) : :- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(i, j, k, l, m)]]]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) +- LogicalTableFunctionScan(invocation=[split($cor0.c)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) ]]> @@ -957,11 +957,11 @@ LegacySink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a, :- Exchange(distribution=[hash[a]], changelogMode=[I]) : +- Join(joinType=[InnerJoin], where=[=(b, e)], select=[a, b, c, d, e, f], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], changelogMode=[I]) : :- Exchange(distribution=[hash[b]], changelogMode=[I]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) : +- Exchange(distribution=[hash[e]], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[d, e, f], changelogMode=[I]) +- Exchange(distribution=[hash[i]], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(i, j, k, l, m)]]], fields=[i, j, k, l, m], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[i, j, k, l, m], changelogMode=[I]) ]]> @@ -973,9 +973,9 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fie +- LogicalProject(a=[$0]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(d=[$0], f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -985,9 +985,9 @@ LegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[to +- Exchange(distribution=[single], changelogMode=[I]) +- Union(all=[true], union=[a], changelogMode=[I]) :- Calc(select=[a], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Calc(select=[d AS a], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[d, e, f], changelogMode=[I]) ]]> @@ -1000,7 +1000,7 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fie +- LogicalProject(a=[$0], b=[$1], c=[$2], rank_num=[$3]) +- LogicalFilter(condition=[<=($3, 10)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rank_num=[ROW_NUMBER() OVER (PARTITION BY $1 ORDER BY $2 DESC NULLS LAST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[a, b]) +- LogicalProject(a=[$0], b=[$1]) @@ -1008,7 +1008,7 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fiel +- LogicalProject(a=[$0], b=[$1], c=[$2], rank_num=[$3]) +- LogicalFilter(condition=[<=($3, 10)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rank_num=[ROW_NUMBER() OVER (PARTITION BY $1 ORDER BY $2 DESC NULLS LAST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1017,13 +1017,13 @@ LegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[a] +- Calc(select=[a], where=[>(a, 6)], changelogMode=[I,UB,UA,D]) +- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=10], partitionBy=[b], orderBy=[c DESC], select=[a, b, c, w0$o0], changelogMode=[I,UB,UA,D]) +- Exchange(distribution=[hash[b]], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) LegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[a, b], changelogMode=[NONE]) +- Calc(select=[a, b], where=[<(a, 6)], changelogMode=[I,UB,UA,D]) +- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=10], partitionBy=[b], orderBy=[c DESC], select=[a, b, c, w0$o0], changelogMode=[I,UB,UA,D]) +- Exchange(distribution=[hash[b]], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -1034,11 +1034,11 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fiel +- LogicalUnion(all=[true]) :- LogicalAggregate(group=[{0, 1}], a_sum=[SUM($2)]) : +- LogicalProject(b=[$1], c=[$2], a=[$0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(b=[1], c=[$0], a_sum=[$1]) +- LogicalAggregate(group=[{0}], a_sum=[SUM($1)]) +- LogicalProject(c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1047,12 +1047,12 @@ LegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[b, +- Union(all=[true], union=[b, c, a_sum], changelogMode=[I,UA]) :- GroupAggregate(groupBy=[b, c], select=[b, c, SUM(a) AS a_sum], changelogMode=[I,UA]) : +- Exchange(distribution=[hash[b, c]], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) +- Calc(select=[CAST(1 AS BIGINT) AS b, c, a_sum], changelogMode=[I,UA]) +- GroupAggregate(groupBy=[c], select=[c, SUM(a) AS a_sum], changelogMode=[I,UA]) +- Exchange(distribution=[hash[c]], changelogMode=[I]) +- Calc(select=[c, a], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -1064,13 +1064,13 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fie +- LogicalFilter(condition=[<=($3, 10)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rank_num=[ROW_NUMBER() OVER (PARTITION BY $1 ORDER BY $2 DESC NULLS LAST)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) LogicalLegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[a, b]) +- LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[<($0, 6)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1078,11 +1078,11 @@ LogicalLegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fiel LegacySink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[a, b, c, rank_num], changelogMode=[NONE]) +- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=10], partitionBy=[b], orderBy=[c DESC], select=[a, b, c, w0$o0], changelogMode=[I,UB,UA,D]) +- Exchange(distribution=[hash[b]], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) LegacySink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[a, b], changelogMode=[NONE]) +- Calc(select=[a, b], where=[<(a, 6)], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.xml index a0a00b51d832b..b28f0c7aacaf4 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.xml @@ -34,7 +34,7 @@ LogicalSink(table=[default_catalog.default_database.sink_with_pk], fields=[a, cn +- LogicalProject(a=[$0], cnt=[$1], valid_uv=[CAST($2):VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL]) +- LogicalAggregate(group=[{0}], cnt=[COUNT()], agg#1=[COUNT(DISTINCT $1) FILTER $2]) +- LogicalProject(a=[$0], c=[$2], $f2=[IS TRUE(>($1, -(UNIX_TIMESTAMP(), 180)))]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -44,7 +44,7 @@ Sink(table=[default_catalog.default_database.sink_with_pk], fields=[a, cnt, vali +- GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt, COUNT(DISTINCT c) FILTER $f2 AS $f2]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, c, (b > (UNIX_TIMESTAMP() - 180)) IS TRUE AS $f2]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d]) ]]> @@ -66,7 +66,7 @@ LogicalSink(table=[default_catalog.default_database.sink_with_pk], fields=[a, cn +- LogicalProject(a=[$0], cnt=[$1], valid_uv=[CAST($2):VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL]) +- LogicalAggregate(group=[{0}], cnt=[COUNT()], agg#1=[COUNT(DISTINCT $1) FILTER $2]) +- LogicalProject(a=[$0], c=[$2], $f2=[IS TRUE(>($1, -(UNIX_TIMESTAMP(), 180)))]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -76,7 +76,7 @@ Sink(table=[default_catalog.default_database.sink_with_pk], fields=[a, cnt, vali +- GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt, COUNT(DISTINCT c) FILTER $f2 AS $f2]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, c, (b > (UNIX_TIMESTAMP() - 180)) IS TRUE AS $f2]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d]) ]]> @@ -1896,7 +1896,7 @@ from T LogicalSink(table=[default_catalog.default_database.sink_without_pk], fields=[EXPR$0, ndCnt, mc]) +- LogicalAggregate(group=[{}], EXPR$0=[MAX($0)], ndCnt=[ndAggFunc($1)], mc=[MAX($2)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -1905,7 +1905,7 @@ Sink(table=[default_catalog.default_database.sink_without_pk], fields=[EXPR$0, n +- GroupAggregate(select=[MAX(a) AS EXPR$0, ndAggFunc(b) AS ndCnt, MAX(c) AS mc]) +- Exchange(distribution=[single]) +- Calc(select=[a, b, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d]) ]]> @@ -1925,7 +1925,7 @@ from T LogicalSink(table=[default_catalog.default_database.sink_with_pk], fields=[EXPR$0, ndCnt, mc]) +- LogicalAggregate(group=[{}], EXPR$0=[MAX($0)], ndCnt=[ndAggFunc($1)], mc=[MAX($2)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -1934,7 +1934,7 @@ Sink(table=[default_catalog.default_database.sink_with_pk], fields=[EXPR$0, ndCn +- GroupAggregate(select=[MAX(a) AS EXPR$0, ndAggFunc(b) AS ndCnt, MAX(c) AS mc]) +- Exchange(distribution=[single]) +- Calc(select=[a, b, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d]) ]]> @@ -1954,7 +1954,7 @@ from T LogicalSink(table=[default_catalog.default_database.sink_with_pk], fields=[EXPR$0, ndCnt, mc]) +- LogicalAggregate(group=[{}], EXPR$0=[MAX($0)], ndCnt=[ndAggFunc($1)], mc=[MAX($2)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -1963,7 +1963,7 @@ Sink(table=[default_catalog.default_database.sink_with_pk], fields=[EXPR$0, ndCn +- GroupAggregate(select=[MAX(a) AS EXPR$0, ndAggFunc(b) AS ndCnt, MAX(c) AS mc]) +- Exchange(distribution=[single]) +- Calc(select=[a, b, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d]) ]]> @@ -2579,7 +2579,7 @@ group by a LogicalSink(table=[default_catalog.default_database.sink_without_pk], fields=[a, ndCnt, mc]) +- LogicalAggregate(group=[{0}], ndCnt=[ndAggFunc($1)], mc=[MAX($2)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -2588,7 +2588,7 @@ Sink(table=[default_catalog.default_database.sink_without_pk], fields=[a, ndCnt, +- GroupAggregate(groupBy=[a], select=[a, ndAggFunc(b) AS ndCnt, MAX(c) AS mc]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d]) ]]> @@ -2609,7 +2609,7 @@ group by a LogicalSink(table=[default_catalog.default_database.sink_with_pk], fields=[a, ndCnt, mc]) +- LogicalAggregate(group=[{0}], ndCnt=[ndAggFunc($1)], mc=[MAX($2)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -2618,7 +2618,7 @@ Sink(table=[default_catalog.default_database.sink_with_pk], fields=[a, ndCnt, mc +- GroupAggregate(groupBy=[a], select=[a, ndAggFunc(b) AS ndCnt, MAX(c) AS mc]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d]) ]]> @@ -2639,7 +2639,7 @@ group by a LogicalSink(table=[default_catalog.default_database.sink_with_pk], fields=[a, ndCnt, mc]) +- LogicalAggregate(group=[{0}], ndCnt=[ndAggFunc($1)], mc=[MAX($2)]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -2648,7 +2648,7 @@ Sink(table=[default_catalog.default_database.sink_with_pk], fields=[a, ndCnt, mc +- GroupAggregate(groupBy=[a], select=[a, ndAggFunc(b) AS ndCnt, MAX(c) AS mc]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.xml index cdb07a19f8227..47cafef7a68c4 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.xml @@ -24,13 +24,13 @@ limitations under the License. @@ -42,14 +42,14 @@ Sink(table=[default_catalog.default_database.sink], fields=[a, b, c]) @@ -61,14 +61,14 @@ Sink(table=[default_catalog.default_database.sinkShuffleBy], fields=[a, b, c]) @@ -80,14 +80,14 @@ Sink(table=[default_catalog.default_database.sink], fields=[a, EXPR$1, c]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/RankTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/RankTest.xml index f6abe90d2d0f4..af37f7294d1af 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/RankTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/RankTest.xml @@ -1137,7 +1137,7 @@ LogicalProject(category=[$0], shopId=[$1], max_price=[$2], row_num=[$3]) +- LogicalFilter(condition=[<=($3, 3)]) +- LogicalProject(category=[$0], shopId=[$1], max_price=[$2], row_num=[ROW_NUMBER() OVER (PARTITION BY $0 ORDER BY $2 NULLS FIRST)]) +- LogicalAggregate(group=[{0, 1}], max_price=[MAX($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(category, shopId, price)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -1146,7 +1146,7 @@ Rank(strategy=[RetractStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, +- Exchange(distribution=[hash[category]], changelogMode=[I,UB,UA]) +- GroupAggregate(groupBy=[category, shopId], select=[category, shopId, MAX(price) AS max_price], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[category, shopId]], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(category, shopId, price)]]], fields=[category, shopId, price], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[category, shopId, price], changelogMode=[I]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.xml index 61552b112b272..0a190d8d5f806 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.xml @@ -24,9 +24,9 @@ limitations under the License. @@ -36,10 +36,10 @@ GroupAggregate(groupBy=[c], select=[c]) +- Join(joinType=[LeftSemiJoin], where=[IS NOT DISTINCT FROM(c, f)], select=[c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[c]]) : +- Calc(select=[c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a, b, c]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[f]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[d, e, f]) ]]> @@ -51,9 +51,9 @@ GroupAggregate(groupBy=[c], select=[c]) @@ -65,9 +65,9 @@ Calc(select=[c0 AS c]) +- Exchange(distribution=[hash[c]]) +- Union(all=[true], union=[c, vcol_left_marker, vcol_right_marker]) :- Calc(select=[c, true AS vcol_left_marker, null:BOOLEAN AS vcol_right_marker]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a, b, c]) +- Calc(select=[f, null:BOOLEAN AS vcol_left_marker, true AS vcol_right_marker]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[d, e, f]) ]]> @@ -80,9 +80,9 @@ Calc(select=[c0 AS c]) LogicalIntersect(all=[false]) :- LogicalProject(c=[$2]) : +- LogicalFilter(condition=[=(1, 0)]) -: +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -99,10 +99,10 @@ Values(tuples=[[]]) @@ -120,9 +120,9 @@ Values(tuples=[[]]) LogicalProject(a=[$0]) +- LogicalIntersect(all=[false]) :- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(d=[$0], e=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -133,10 +133,10 @@ Calc(select=[a]) +- Join(joinType=[LeftSemiJoin], where=[(((a = d) OR (a IS NULL AND d IS NULL)) AND ((b = e) OR (b IS NULL AND e IS NULL)))], select=[a, b], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a, b]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, e]]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[d, e, f]) ]]> @@ -148,9 +148,9 @@ Calc(select=[a]) @@ -160,10 +160,10 @@ GroupAggregate(groupBy=[c], select=[c]) +- Join(joinType=[LeftAntiJoin], where=[IS NOT DISTINCT FROM(c, f)], select=[c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[c]]) : +- Calc(select=[c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a, b, c]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[f]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[d, e, f]) ]]> @@ -175,9 +175,9 @@ GroupAggregate(groupBy=[c], select=[c]) @@ -189,9 +189,9 @@ Calc(select=[c0 AS c]) +- Exchange(distribution=[hash[c]]) +- Union(all=[true], union=[c, vcol_marker]) :- Calc(select=[c, 1 AS vcol_marker]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a, b, c]) +- Calc(select=[f, -1 AS vcol_marker]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[d, e, f]) ]]> @@ -204,9 +204,9 @@ Calc(select=[c0 AS c]) LogicalMinus(all=[false]) :- LogicalProject(c=[$2]) : +- LogicalFilter(condition=[=(1, 0)]) -: +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(f=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -223,10 +223,10 @@ Values(tuples=[[]]) @@ -234,7 +234,7 @@ LogicalMinus(all=[false]) GroupAggregate(groupBy=[c], select=[c]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a, b, c]) ]]> @@ -246,9 +246,9 @@ GroupAggregate(groupBy=[c], select=[c]) @@ -257,7 +257,7 @@ GroupAggregate(groupBy=[a, b, c], select=[a, b, c]) +- Exchange(distribution=[hash[a, b, c]]) +- Join(joinType=[LeftAntiJoin], where=[(((a = a0) OR (a IS NULL AND a0 IS NULL)) AND ((b = b0) OR (b IS NULL AND b0 IS NULL)) AND ((c = c0) OR (c IS NULL AND c0 IS NULL)))], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a, b, c]])(reuse_id=[1]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) +- Reused(reference_id=[1]) ]]> @@ -270,16 +270,16 @@ GroupAggregate(groupBy=[a, b, c], select=[a, b, c]) ($2, 0), $1, null:RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) CHARACTER SET "UTF-16LE" _2))]) - +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, A]]) ]]> 0), b, null:RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2)) AS EXPR$0]) +- Reused(reference_id=[1]) ]]> @@ -293,16 +293,16 @@ Union(all=[true], union=[a]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.xml index c6979ac3d0cef..e1f9644f71ed8 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.xml @@ -29,17 +29,17 @@ UNION ALL LogicalUnion(all=[true]) :- LogicalProject(a=[$0]) : +- LogicalFilter(condition=[>($1, random_udf($0))]) -: +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0]) +- LogicalFilter(condition=[>($1, random_udf($0))]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> random_udf(a))])(reuse_id=[1]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Reused(reference_id=[1]) ]]> @@ -59,13 +59,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], a0=[$6], b0=[$7], :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) : +- LogicalFilter(condition=[=($0, $3)]) : +- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[=($0, $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -75,18 +75,18 @@ Join(joinType=[InnerJoin], where=[(b = e0)], select=[a, b, c, d, e, f, a0, b0, c : +- Join(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) : :- Exchange(distribution=[hash[a]]) : : +- Calc(select=[a, b, c], where=[(a < 10)]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, e, f], where=[(d < 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) +- Exchange(distribution=[hash[e]]) +- Join(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[(a > 5)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, e, f], where=[(d > 5)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) ]]> @@ -154,13 +154,13 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3], a0=[$4], b0=[$5], d0=[$6], e0=[$7 :- LogicalProject(a=[$0], b=[$1], d=[$3], e=[$4]) : +- LogicalFilter(condition=[=($0, $3)]) : +- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[$1], d=[$3], e=[$4]) +- LogicalFilter(condition=[=($0, $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -170,10 +170,10 @@ Join(joinType=[InnerJoin], where=[(b = e0)], select=[a, b, d, e, a0, b0, d0, e0] : +- Join(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) : :- Exchange(distribution=[hash[a]]) : : +- Calc(select=[a, b], where=[(a < 10)]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])(reuse_id=[1]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c])(reuse_id=[1]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, e], where=[(d < 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])(reuse_id=[2]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f])(reuse_id=[2]) +- Exchange(distribution=[hash[e]]) +- Join(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) @@ -372,10 +372,10 @@ LogicalProject(c=[$0], a=[$1], b=[$2], c0=[$3], a0=[$4], b0=[$5]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalAggregate(group=[{0}], a=[MyFirst($1)], b=[MyLast($2)]) : +- LogicalProject(c=[$2], a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalAggregate(group=[{0}], a=[MyFirst($1)], b=[MyLast($2)]) +- LogicalProject(c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -386,7 +386,7 @@ Calc(select=[c, a, b, c0, a1, b0]) : +- Calc(select=[c, a, b, CAST(a AS BIGINT) AS a0]) : +- GroupAggregate(groupBy=[c], select=[c, MyFirst(a) AS a, MyLast(b) AS b])(reuse_id=[1]) : +- Exchange(distribution=[hash[c]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[c, a, b], where=[(a > 1)]) +- Reused(reference_id=[1]) @@ -410,15 +410,15 @@ LogicalUnion(all=[true]) : +- LogicalJoin(condition=[true], joinType=[inner]) : :- LogicalProject(a=[$0], b=[$1], c=[$2]) : : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'test%')]) -: : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) -: +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) +: : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], c=[LOWER($2)], e=[$4]) +- LogicalFilter(condition=[=($0, $3)]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'test%')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -428,10 +428,10 @@ Union(all=[true], union=[a, c, e]) : +- Join(joinType=[InnerJoin], where=[(a = d)], select=[a, c, d, e], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) : :- Exchange(distribution=[hash[a]]) : : +- Calc(select=[a, c], where=[LIKE(c, 'test%')]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, e]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) +- Reused(reference_id=[1]) ]]> @@ -449,17 +449,17 @@ UNION ALL LogicalUnion(all=[true]) :- LogicalProject(a=[$0], EXPR$1=[random_udf()]) : +- LogicalFilter(condition=[>($0, 10)]) -: +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0], EXPR$1=[random_udf()]) +- LogicalFilter(condition=[>($0, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> 10)])(reuse_id=[1]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Reused(reference_id=[1]) ]]> @@ -506,11 +506,11 @@ LogicalProject(a=[$0], b=[$1], c=[$2], v=[$3], a0=[$4], b0=[$5], c0=[$6], v0=[$7 +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], v=[$3]) : +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalTableFunctionScan(invocation=[str_split($cor0.c, _UTF-16LE'-')], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], v=[$3]) +- LogicalCorrelate(correlation=[$cor1], joinType=[inner], requiredColumns=[{2}]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalTableFunctionScan(invocation=[str_split($cor1.c, _UTF-16LE'-')], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) ]]> @@ -519,10 +519,10 @@ LogicalProject(a=[$0], b=[$1], c=[$2], v=[$3], a0=[$4], b0=[$5], c0=[$6], v0=[$7 Join(joinType=[InnerJoin], where=[(EXPR$0 = EXPR$00)], select=[a, b, c, EXPR$0, a0, b0, c0, EXPR$00], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[EXPR$0]]) : +- Correlate(invocation=[str_split($cor0.c, _UTF-16LE'-')], correlate=[table(str_split($cor0.c,'-'))], select=[a,b,c,EXPR$0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) EXPR$0)], joinType=[INNER]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[EXPR$0]]) +- Correlate(invocation=[str_split($cor1.c, _UTF-16LE'-')], correlate=[table(str_split($cor1.c,'-'))], select=[a,b,c,EXPR$0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) EXPR$0)], joinType=[INNER]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) ]]> @@ -540,11 +540,11 @@ LogicalProject(a=[$0], b=[$1], c=[$2], s=[$3], a0=[$4], b0=[$5], c0=[$6], s0=[$7 +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], s=[$3]) : +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - : :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : :- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalTableFunctionScan(invocation=[TableFun($cor0.c)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor1], joinType=[inner], requiredColumns=[{2}]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalTableFunctionScan(invocation=[TableFun($cor1.c)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) ]]> @@ -553,10 +553,10 @@ LogicalProject(a=[$0], b=[$1], c=[$2], s=[$3], a0=[$4], b0=[$5], c0=[$6], s0=[$7 Join(joinType=[InnerJoin], where=[(c = EXPR$00)], select=[a, b, c, EXPR$0, a0, b0, c0, EXPR$00], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[c]]) : +- Correlate(invocation=[TableFun($cor0.c)], correlate=[table(TableFun($cor0.c))], select=[a,b,c,EXPR$0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) EXPR$0)], joinType=[INNER]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[EXPR$0]]) +- Correlate(invocation=[TableFun($cor1.c)], correlate=[table(TableFun($cor1.c))], select=[a,b,c,EXPR$0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) EXPR$0)], joinType=[INNER]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) ]]> @@ -577,15 +577,15 @@ LogicalUnion(all=[true]) : +- LogicalJoin(condition=[true], joinType=[inner]) : :- LogicalProject(a=[$0], b=[$1], c=[$2]) : : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'test%')]) -: : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) -: +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) +: : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), <>($5, _UTF-16LE''))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'test%')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -594,15 +594,15 @@ Union(all=[true], union=[a, b, c, d, e, f]) :- Join(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) : :- Exchange(distribution=[hash[a]])(reuse_id=[1]) : : +- Calc(select=[a, b, c], where=[LIKE(c, 'test%')]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d, e, f], where=[(e > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) +- Join(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Reused(reference_id=[1]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, e, f], where=[(f <> '')]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) ]]> @@ -620,10 +620,10 @@ LogicalProject(c=[$0], a=[$1], b=[$2], c0=[$3], a0=[$4], b0=[$5]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalAggregate(group=[{0}], a=[SUM($1)], b=[SUM($2)]) : +- LogicalProject(c=[$2], a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalAggregate(group=[{0}], a=[SUM($1)], b=[SUM($2)]) +- LogicalProject(c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -634,7 +634,7 @@ Calc(select=[c, a, b, c0, a1, b0]) : +- Calc(select=[c, a, b, CAST(a AS BIGINT) AS a0]) : +- GroupAggregate(groupBy=[c], select=[c, SUM(a) AS a, SUM(b) AS b])(reuse_id=[1]) : +- Exchange(distribution=[hash[c]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[c, a, b], where=[(a > 1)]) +- Reused(reference_id=[1]) @@ -659,14 +659,14 @@ LogicalUnion(all=[true]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) : +- LogicalFilter(condition=[AND(>($1, 1), <($4, 2))]) : +- LogicalJoin(condition=[OR(=(ABS($0), ABS($3)), =($2, $5))], joinType=[full]) -: :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) -: +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) +: :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[*($1, 2)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(>($1, 1), <($4, 2))]) +- LogicalJoin(condition=[OR(=(ABS($0), ABS($3)), =($2, $5))], joinType=[full]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -676,10 +676,10 @@ Union(all=[true], union=[a, b]) : +- Join(joinType=[InnerJoin], where=[((ABS(a) = ABS(d)) OR (c = f))], select=[a, b, c, d, f], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])(reuse_id=[1]) : :- Exchange(distribution=[single]) : : +- Calc(select=[a, b, c], where=[(b > 1)]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) : +- Exchange(distribution=[single]) : +- Calc(select=[d, f], where=[(e < 2)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) +- Calc(select=[a, (b * 2) AS b]) +- Reused(reference_id=[1]) ]]> @@ -703,14 +703,14 @@ LogicalUnion(all=[true]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) : +- LogicalFilter(condition=[AND(>($1, 1), <($4, 2))]) : +- LogicalJoin(condition=[OR(=(random_udf($0), random_udf($3)), =($2, $5))], joinType=[full]) -: :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) -: +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) +: :- LogicalTableScan(table=[[default_catalog, default_database, x]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, y]]) +- LogicalProject(a=[$0], b=[*($1, 2)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(>($1, 1), <($4, 2))]) +- LogicalJoin(condition=[OR(=(random_udf($0), random_udf($3)), =($2, $5))], joinType=[full]) - :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, y]]) ]]> @@ -720,10 +720,10 @@ Union(all=[true], union=[a, b]) : +- Join(joinType=[InnerJoin], where=[((random_udf(a) = random_udf(d)) OR (c = f))], select=[a, b, c, d, f], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])(reuse_id=[1]) : :- Exchange(distribution=[single]) : : +- Calc(select=[a, b, c], where=[(b > 1)]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) : +- Exchange(distribution=[single]) : +- Calc(select=[d, f], where=[(e < 2)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, y]], fields=[d, e, f]) +- Calc(select=[a, (b * 2) AS b]) +- Reused(reference_id=[1]) ]]> @@ -742,9 +742,9 @@ LogicalProject(a=[$0], b=[$1], EXPR$2=[$2], a0=[$3], b0=[$4], EXPR$20=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), <($1, 100), >($4, 10))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], EXPR$2=[RANK() OVER (ORDER BY $2 DESC NULLS LAST)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0], b=[$1], EXPR$2=[RANK() OVER (ORDER BY $2 DESC NULLS LAST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -754,7 +754,7 @@ Join(joinType=[InnerJoin], where=[(a = a0)], select=[a, b, w0$o0, a0, b0, w0$o00 : +- Calc(select=[a, b, w0$o0], where=[(b < 100)]) : +- OverAggregate(orderBy=[c DESC], window=[ RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, RANK(*) AS w0$o0])(reuse_id=[1]) : +- Exchange(distribution=[single]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, w0$o0], where=[(b > 10)]) +- Reused(reference_id=[1]) @@ -778,12 +778,12 @@ LogicalUnion(all=[true]) : +- LogicalFilter(condition=[>($0, 10)]) : +- LogicalSort(fetch=[10]) : +- LogicalProject(a=[$0], b=[$1]) -: +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0], b=[*($1, 2)]) +- LogicalFilter(condition=[<($1, 10)]) +- LogicalSort(fetch=[10]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -792,7 +792,7 @@ Union(all=[true], union=[a, b]) :- Calc(select=[a, b], where=[(a > 10)]) : +- Limit(offset=[0], fetch=[10])(reuse_id=[1]) : +- Exchange(distribution=[single]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Calc(select=[a, (b * 2) AS b], where=[(b < 10)]) +- Reused(reference_id=[1]) ]]> @@ -811,9 +811,9 @@ LogicalProject(a=[$0], b=[$1], EXPR$2=[$2], a0=[$3], b0=[$4], EXPR$20=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), <($1, 100), >($4, 10))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], EXPR$2=[MyFirst($2) OVER (PARTITION BY $2 ORDER BY $2 DESC NULLS LAST)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$0], b=[$1], EXPR$2=[MyFirst($2) OVER (PARTITION BY $2 ORDER BY $2 DESC NULLS LAST)]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -823,7 +823,7 @@ Join(joinType=[InnerJoin], where=[(a = a0)], select=[a, b, w0$o0, a0, b0, w0$o00 : +- Calc(select=[a, b, w0$o0], where=[(b < 100)]) : +- OverAggregate(partitionBy=[c], orderBy=[c DESC], window=[ RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, MyFirst(c) AS w0$o0])(reuse_id=[1]) : +- Exchange(distribution=[hash[c]]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, w0$o0], where=[(b > 10)]) +- Reused(reference_id=[1]) @@ -844,10 +844,10 @@ LogicalProject(c=[$0], a=[$1], b=[$2], c0=[$3], a0=[$4], b0=[$5]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalAggregate(group=[{0}], a=[SUM($1)], b=[SUM($2)]) : +- LogicalProject(c=[$2], a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalAggregate(group=[{0}], a=[SUM($1)], b=[SUM($2)]) +- LogicalProject(c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -857,7 +857,7 @@ Join(joinType=[InnerJoin], where=[(a = a0)], select=[c, a, b, c0, a0, b0], leftI : +- Calc(select=[c, a, b], where=[(a > 1)]) : +- GroupAggregate(groupBy=[c], select=[c, SUM(a) AS a, SUM(b) AS b])(reuse_id=[1]) : +- Exchange(distribution=[hash[c]]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[c, a, b], where=[((b < 10) AND (a > 1))]) +- Reused(reference_id=[1]) @@ -882,13 +882,13 @@ LogicalUnion(all=[true]) : +- LogicalSort(sort0=[$1], sort1=[$2], dir0=[ASC-nulls-first], dir1=[DESC-nulls-last], fetch=[10]) : +- LogicalAggregate(group=[{0}], a=[SUM($1)], b=[SUM($2)]) : +- LogicalProject(c=[$2], a=[$0], b=[$1]) -: +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(a=[$1], b=[*($2, 2)]) +- LogicalFilter(condition=[<($2, 10)]) +- LogicalSort(sort0=[$1], sort1=[$2], dir0=[ASC-nulls-first], dir1=[DESC-nulls-last], fetch=[10]) +- LogicalAggregate(group=[{0}], a=[SUM($1)], b=[SUM($2)]) +- LogicalProject(c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -899,7 +899,7 @@ Union(all=[true], union=[a, b]) : +- Exchange(distribution=[single]) : +- GroupAggregate(groupBy=[c], select=[c, SUM(a) AS a, SUM(b) AS b]) : +- Exchange(distribution=[hash[c]]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Calc(select=[a, (b * 2) AS b], where=[(b < 10)]) +- Reused(reference_id=[1]) ]]> @@ -913,15 +913,15 @@ LogicalIntersect(all=[false]) : :- LogicalProject(random=[$0]) : : +- LogicalSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[1]) : : +- LogicalProject(random=[$0], EXPR$1=[RAND()]) -: : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) +: : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) : +- LogicalProject(random=[$0]) : +- LogicalSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[1]) : +- LogicalProject(random=[$0], EXPR$1=[RAND()]) -: +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) +: +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(random=[$0]) +- LogicalSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[1]) +- LogicalProject(random=[$0], EXPR$1=[RAND()]) - +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -936,7 +936,7 @@ Join(joinType=[LeftSemiJoin], where=[IS NOT DISTINCT FROM(random, random0)], sel : : +- SortLimit(orderBy=[EXPR$1 ASC], offset=[0], fetch=[1], strategy=[AppendFastStrategy]) : : +- Exchange(distribution=[single]) : : +- Calc(select=[a AS random, RAND() AS EXPR$1]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) : +- Reused(reference_id=[1]) +- Reused(reference_id=[1]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml index 465c4b0c67ce5..95ba887b90edc 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml @@ -455,22 +455,6 @@ LogicalProject(a=[$0], b=[$1], time=[$2], mytime=[$3], current_time=[$4], json_r WatermarkAssigner(rowtime=[timestamp], watermark=[timestamp]) +- Calc(select=[a, b, time, time AS mytime, CURRENT_TIME() AS current_time, json_row, json_row.timestamp AS timestamp]) +- TableSourceScan(table=[[default_catalog, default_database, t1]], fields=[a, b, time, json_row]) -]]> - - - - - - - - - - - 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 7997be7984bfe..db3012d786833 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 @@ -71,11 +71,11 @@ LogicalProject(a=[$0], c=[$1]) +- LogicalUnion(all=[true]) :- LogicalUnion(all=[true]) : :- LogicalProject(a=[$0], c=[$2]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) : +- LogicalProject(a=[$0], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) +- LogicalProject(a=[$0], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3]]) ]]> @@ -83,11 +83,11 @@ LogicalProject(a=[$0], c=[$1]) Union(all=[true], union=[a, c]) :- Union(all=[true], union=[a, c]) : :- Calc(select=[a, c], where=[(a > 2)]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) : +- Calc(select=[a, c], where=[(a > 2)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c]) +- Calc(select=[a, c], where=[(a > 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable3]], fields=[a, b, c]) ]]> @@ -105,18 +105,18 @@ SELECT * FROM ( 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, source: [TestTableSource(a, b, c)]]]), rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)] + : +- 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)] - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]]), rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)] + +- 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/UnnestTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml index 273204ae4157c..dcfcc0c4e3b40 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml @@ -43,7 +43,7 @@ Correlate(invocation=[$UNNEST_ROWS$1(ARRAY(1, 2, 3))], correlate=[table($UNNEST_ @@ -66,7 +66,7 @@ Calc(select=[a, f0 AS s]) @@ -97,7 +97,7 @@ LogicalProject(a=[$0], b=[$1], x=[$2], y=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalProject(a=[$0], b=[$1]) : +- LogicalFilter(condition=[<($0, 3)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(x=[$0], y=[$1]) +- Uncollect +- LogicalProject(b=[$cor0.b]) @@ -109,7 +109,7 @@ LogicalProject(a=[$0], b=[$1], x=[$2], y=[$3]) Calc(select=[a, b, _1, _2], where=[(_1 > a)]) +- Correlate(invocation=[$UNNEST_ROWS$1($cor0.b)], correlate=[table($UNNEST_ROWS$1($cor0.b))], select=[a,b,_1,_2], rowType=[RecordType(INTEGER a, RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2) ARRAY b, INTEGER _1, VARCHAR(2147483647) _2)], joinType=[INNER]) +- Calc(select=[a, b], where=[(a < 3)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]], fields=[a, b]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> @@ -157,7 +157,7 @@ FROM T2 CROSS JOIN UNNEST(T2.b) AS D(c) @@ -216,7 +216,7 @@ Calc(select=[b, f0 AS s]) @@ -272,7 +272,7 @@ Calc(select=[b, _1 AS id, _2 AS point]) LogicalProject(a=[$0], b=[$1], _1=[$2], _2=[$3]) +- LogicalFilter(condition=[>($2, 1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- Uncollect +- LogicalProject(b=[$cor0.b]) +- LogicalValues(tuples=[[{ 0 }]]) @@ -281,7 +281,7 @@ LogicalProject(a=[$0], b=[$1], _1=[$2], _2=[$3]) ($0, 1)]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]], fields=[a, b]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> @@ -293,7 +293,7 @@ Correlate(invocation=[$UNNEST_ROWS$1($cor0.b)], correlate=[table($UNNEST_ROWS$1( @@ -317,7 +317,7 @@ Calc(select=[a, b, f0 AS s]) LogicalProject(a=[$0], b=[$1], s=[$2], t=[$3]) +- LogicalFilter(condition=[>($2, 13)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(s=[$0], t=[$1]) +- Uncollect +- LogicalProject(b=[$cor0.b]) @@ -327,7 +327,7 @@ LogicalProject(a=[$0], b=[$1], s=[$2], t=[$3]) ($0, 13)]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]], fields=[a, b]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> @@ -352,7 +352,7 @@ LogicalProject(a=[$0], b1=[$1], b2=[$2]) +- LogicalFilter(condition=[>=($2, 12)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalProject(a=[$0], b=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(b1=[$0], b2=[$1]) +- Uncollect +- LogicalProject(b=[$cor0.b]) @@ -363,7 +363,7 @@ LogicalProject(a=[$0], b1=[$1], b2=[$2]) =($0, 12), <>($1, _UTF-16LE'Hello'))]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]], fields=[a, b]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b]) ]]> 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 4a36ab4518649..47ccb1cd87451 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 @@ -32,9 +32,9 @@ FROM ( LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT(DISTINCT $2)]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -44,9 +44,9 @@ GlobalGroupAggregate(groupBy=[a], select=[a, SUM(sum$0) AS EXPR$1, COUNT(distinc +- LocalGroupAggregate(groupBy=[a], select=[a, SUM(b) AS sum$0, COUNT(distinct$0 c) AS count$1, DISTINCT(c) AS distinct$0]) +- Union(all=[true], union=[a, b, c]) :- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a, b, c]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a, b, c]) ]]> @@ -69,7 +69,7 @@ FROM T GROUP BY a @@ -77,7 +77,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[SUM($2)], EXPR$3=[SUM($3 GroupAggregate(groupBy=[a], select=[a, SUM($f1) AS EXPR$1, SUM($f2) AS EXPR$2, SUM($f3) AS EXPR$3, SUM($f4) AS EXPR$4, SUM($f5) AS EXPR$5, SUM($f6) AS EXPR$6, SUM($f7) AS EXPR$7]), rowType=[RecordType(INTEGER a, INTEGER EXPR$1, BIGINT EXPR$2, TINYINT EXPR$3, SMALLINT EXPR$4, FLOAT EXPR$5, DECIMAL(38, 0) EXPR$6, DOUBLE EXPR$7)] +- Exchange(distribution=[hash[a]]), rowType=[RecordType(INTEGER a, INTEGER $f1, BIGINT $f2, TINYINT $f3, SMALLINT $f4, FLOAT $f5, DECIMAL(10, 0) $f6, DOUBLE $f7)] +- Calc(select=[a, 1 AS $f1, 2 AS $f2, 3 AS $f3, 4 AS $f4, 5 AS $f5, 6 AS $f6, 7 AS $f7]), rowType=[RecordType(INTEGER a, INTEGER $f1, BIGINT $f2, TINYINT $f3, SMALLINT $f4, FLOAT $f5, DECIMAL(10, 0) $f6, DOUBLE $f7)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]), rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, BOOLEAN d)] + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d]), rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, BOOLEAN d)] ]]> @@ -96,7 +96,7 @@ FROM T GROUP BY a @@ -104,7 +104,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1) FILTER $2], EXPR$2=[COUNT(DISTINCT GroupAggregate(groupBy=[a], select=[a, SUM(b) FILTER $f2 AS EXPR$1, COUNT(DISTINCT c) FILTER $f4 AS EXPR$2, MAX(b) AS EXPR$3]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, (c = 'A') IS TRUE AS $f2, c, d IS TRUE AS $f4]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d]) ]]> @@ -124,7 +124,7 @@ FROM T GROUP BY a @@ -134,7 +134,7 @@ GlobalGroupAggregate(groupBy=[a], select=[a, SUM(sum$0) AS EXPR$1, COUNT(distinc +- LocalGroupAggregate(groupBy=[a], select=[a, SUM(b) FILTER $f2 AS sum$0, COUNT(distinct$0 c) FILTER $f4 AS count$1, COUNT(distinct$0 c) FILTER $f5 AS count$2, MAX(b) AS max$3, DISTINCT(c) AS distinct$0]) +- Calc(select=[a, b, (c = 'A') IS TRUE AS $f2, c, d IS TRUE AS $f4, (b = 1) IS TRUE AS $f5]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d]) ]]> @@ -143,7 +143,9 @@ GlobalGroupAggregate(groupBy=[a], select=[a, SUM(sum$0) AS EXPR$1, COUNT(distinc @@ -181,7 +187,7 @@ FROM MyTable1 @@ -189,7 +195,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[AVG($1)], EXPR$2=[AVG($2) GroupAggregate(select=[AVG(byte) AS EXPR$0, AVG(short) AS EXPR$1, AVG(int) AS EXPR$2, AVG(long) AS EXPR$3, AVG(float) AS EXPR$4, AVG(double) AS EXPR$5, AVG(decimal3020) AS EXPR$6, AVG(decimal105) AS EXPR$7]), rowType=[RecordType(TINYINT EXPR$0, SMALLINT EXPR$1, INTEGER EXPR$2, BIGINT EXPR$3, FLOAT EXPR$4, DOUBLE EXPR$5, DECIMAL(38, 20) EXPR$6, DECIMAL(38, 6) EXPR$7)] +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -203,7 +209,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)]) +- LogicalProject(a=[$1]) +- LogicalAggregate(group=[{0}], a=[AVG($1)]) +- LogicalProject(b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -214,7 +220,7 @@ GroupAggregate(select=[AVG_RETRACT(a) AS EXPR$0], changelogMode=[I,UA,D]) +- GroupAggregate(groupBy=[b], select=[b, AVG(a) AS a], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[b]], changelogMode=[I]) +- Calc(select=[b, a], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d], changelogMode=[I]) ]]> @@ -227,7 +233,9 @@ GroupAggregate(select=[AVG_RETRACT(a) AS EXPR$0], changelogMode=[I,UA,D]) LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)]) +- LogicalProject(b=[$1], a=[$0]) +- LogicalFilter(condition=[AND(>($0, 0.1:DECIMAL(2, 1)), <($0, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, proctime, rowtime)]]]) + +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[$4]) + +- LogicalProject(a=[$0], b=[$1], c=[$2], proctime=[PROCTIME()], rowtime=[$3]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -235,7 +243,9 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)]) GroupAggregate(groupBy=[b], select=[b, SUM(a) AS EXPR$1]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[a, b], where=[SEARCH(a, Sarg[(0.1..10)])]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, proctime, rowtime)]]], fields=[a, b, c, proctime, rowtime]) + +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) + +- Calc(select=[a, b, rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, rowtime]) ]]> @@ -341,7 +351,7 @@ LogicalAggregate(group=[{}], all_uv=[SUM($0) FILTER $1]) +- LogicalProject(uv=[$1], $f1=[IS TRUE(=($0, _UTF-16LE'all'))]) +- LogicalAggregate(group=[{0}], uv=[COUNT()]) +- LogicalProject(c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -352,7 +362,7 @@ GroupAggregate(select=[SUM_RETRACT(uv) FILTER $f1 AS all_uv]) +- GroupAggregate(groupBy=[c], select=[c, COUNT(*) AS uv]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d]) ]]> @@ -367,7 +377,7 @@ SELECT a, MAX(b), c FROM (SELECT a, 'test' AS c, b FROM T) t GROUP BY a, c LogicalProject(a=[$0], EXPR$1=[$2], c=[$1]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[MAX($2)]) +- LogicalProject(a=[$0], c=[_UTF-16LE'test'], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -376,7 +386,7 @@ Calc(select=[a, EXPR$1, 'test' AS c]) +- GroupAggregate(groupBy=[a], select=[a, MAX(b) AS EXPR$1]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d]) ]]> @@ -389,7 +399,9 @@ Calc(select=[a, EXPR$1, 'test' AS c]) LogicalProject(EXPR$0=[$1]) +- LogicalAggregate(group=[{0}], EXPR$0=[COUNT($1)]) +- LogicalProject(b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, proctime, rowtime)]]]) + +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[$4]) + +- LogicalProject(a=[$0], b=[$1], c=[$2], proctime=[PROCTIME()], rowtime=[$3]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -398,7 +410,9 @@ Calc(select=[EXPR$0]) +- GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS EXPR$0]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[b, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, proctime, rowtime)]]], fields=[a, b, c, proctime, rowtime]) + +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) + +- Calc(select=[b, a, rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, rowtime]) ]]> @@ -408,21 +422,21 @@ Calc(select=[EXPR$0]) LogicalSink(table=[default_catalog.default_database.sink], fields=[a, b, cnt]) +- LogicalAggregate(group=[{0}], b=[MAX($1)], cnt=[COUNT()]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) == Optimized Physical Plan == Sink(table=[default_catalog.default_database.sink], fields=[a, b, cnt], changelogMode=[NONE]) +- GroupAggregate(groupBy=[a], select=[a, MAX(b) AS b, COUNT(*) AS cnt], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[a]], changelogMode=[I]) +- Calc(select=[a, b], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d], changelogMode=[I]) == Optimized Execution Plan == Sink(table=[default_catalog.default_database.sink], fields=[a, b, cnt]) +- GroupAggregate(groupBy=[a], select=[a, MAX(b) AS b, COUNT(*) AS cnt]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d]) ]]> @@ -432,21 +446,21 @@ Sink(table=[default_catalog.default_database.sink], fields=[a, b, cnt]) LogicalSink(table=[default_catalog.default_database.sink], fields=[c, cnt]) +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) +- LogicalProject(c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) == Optimized Physical Plan == Sink(table=[default_catalog.default_database.sink], fields=[c, cnt], upsertMaterialize=[true], changelogMode=[NONE]) +- GroupAggregate(groupBy=[c], select=[c, COUNT(*) AS cnt], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[c]], changelogMode=[I]) +- Calc(select=[c], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d], changelogMode=[I]) == Optimized Execution Plan == Sink(table=[default_catalog.default_database.sink], fields=[c, cnt], upsertMaterialize=[true]) +- GroupAggregate(groupBy=[c], select=[c, COUNT(*) AS cnt]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d]) ]]> @@ -457,7 +471,7 @@ LogicalSink(table=[default_catalog.default_database.sink], fields=[c, cnt]) +- LogicalProject(c=[$1], cnt=[$2]) +- LogicalAggregate(group=[{0, 1}], cnt=[COUNT()]) +- LogicalProject(a=[$0], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) == Optimized Physical Plan == Sink(table=[default_catalog.default_database.sink], fields=[c, cnt], upsertMaterialize=[true], changelogMode=[NONE]) @@ -465,7 +479,7 @@ Sink(table=[default_catalog.default_database.sink], fields=[c, cnt], upsertMater +- GroupAggregate(groupBy=[a, c], select=[a, c, COUNT(*) AS cnt], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[a, c]], changelogMode=[I]) +- Calc(select=[a, c], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d], changelogMode=[I]) == Optimized Execution Plan == Sink(table=[default_catalog.default_database.sink], fields=[c, cnt], upsertMaterialize=[true]) @@ -473,7 +487,7 @@ Sink(table=[default_catalog.default_database.sink], fields=[c, cnt], upsertMater +- GroupAggregate(groupBy=[a, c], select=[a, c, COUNT(*) AS cnt]) +- Exchange(distribution=[hash[a, c]]) +- Calc(select=[a, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d]) ]]> @@ -493,9 +507,9 @@ FROM ( LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT(DISTINCT $2)]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -505,9 +519,9 @@ GlobalGroupAggregate(groupBy=[a], select=[a, SUM(sum$0) AS EXPR$1, COUNT(distinc +- LocalGroupAggregate(groupBy=[a], select=[a, SUM(b) AS sum$0, COUNT(distinct$0 c) AS count$1, DISTINCT(c) AS distinct$0]) +- Union(all=[true], union=[a, b, c]) :- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a, b, c]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a, b, c]) ]]> @@ -534,14 +548,14 @@ FROM MyTable1 @@ -563,7 +577,7 @@ FROM MyTable1 @@ -571,7 +585,7 @@ LogicalAggregate(group=[{}], EXPR$0=[SUM($0)], EXPR$1=[SUM($1)], EXPR$2=[SUM($2) GroupAggregate(select=[SUM(byte) AS EXPR$0, SUM(short) AS EXPR$1, SUM(int) AS EXPR$2, SUM(long) AS EXPR$3, SUM(float) AS EXPR$4, SUM(double) AS EXPR$5, SUM(decimal3020) AS EXPR$6, SUM(decimal105) AS EXPR$7]), rowType=[RecordType(TINYINT EXPR$0, SMALLINT EXPR$1, INTEGER EXPR$2, BIGINT EXPR$3, FLOAT EXPR$4, DOUBLE EXPR$5, DECIMAL(38, 20) EXPR$6, DECIMAL(38, 5) EXPR$7)] +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable1]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -585,7 +599,7 @@ LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) +- LogicalProject(a=[$1]) +- LogicalAggregate(group=[{0}], a=[MAX($1)]) +- LogicalProject(b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -596,7 +610,7 @@ GroupAggregate(select=[MAX(a) AS EXPR$0], changelogMode=[I,UA,D]) +- GroupAggregate(groupBy=[b], select=[b, MAX(a) AS a], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[b]], changelogMode=[I]) +- Calc(select=[b, a], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d], changelogMode=[I]) ]]> @@ -623,14 +637,14 @@ FROM MyTable1 @@ -644,7 +658,7 @@ LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) +- LogicalProject(a=[$1]) +- LogicalAggregate(group=[{0}], a=[MIN($1)]) +- LogicalProject(b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -655,7 +669,7 @@ GroupAggregate(select=[MIN(a) AS EXPR$0], changelogMode=[I,UA,D]) +- GroupAggregate(groupBy=[b], select=[b, MIN(a) AS a], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[b]], changelogMode=[I]) +- Calc(select=[b, a], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d], changelogMode=[I]) ]]> @@ -669,7 +683,7 @@ LogicalAggregate(group=[{}], EXPR$0=[SUM($0)]) +- LogicalProject(a=[$1]) +- LogicalAggregate(group=[{0}], a=[SUM($1)]) +- LogicalProject(b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T]]) ]]> @@ -680,7 +694,7 @@ GroupAggregate(select=[SUM_RETRACT(a) AS EXPR$0], changelogMode=[I,UA,D]) +- GroupAggregate(groupBy=[b], select=[b, SUM(a) AS a], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[b]], changelogMode=[I]) +- Calc(select=[b, a], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, T]], fields=[a, b, c, d], changelogMode=[I]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.xml index 6ea406b774f1c..9a79c57e503c2 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.xml @@ -32,7 +32,7 @@ GROUP BY a ($1, 2))], $f3=[IS TRUE(<>($1, 5))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -41,7 +41,7 @@ GroupAggregate(groupBy=[a], select=[a, COUNT(DISTINCT b) FILTER $f2 AS EXPR$1, S +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, (b <> 2) IS TRUE AS $f2, (b <> 5) IS TRUE AS $f3]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -61,7 +61,7 @@ GROUP BY a ($1, 2))], $f3=[IS TRUE(<>($1, 5))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -71,7 +71,7 @@ GlobalGroupAggregate(groupBy=[a], select=[a, COUNT(distinct$0 count$0) AS EXPR$1 +- LocalGroupAggregate(groupBy=[a], select=[a, COUNT(distinct$0 b) FILTER $f2 AS count$0, SUM(b) FILTER $f3 AS sum$1, SUM(b) FILTER $f2 AS sum$2, DISTINCT(b) AS distinct$0]) +- Calc(select=[a, b, (b <> 2) IS TRUE AS $f2, (b <> 5) IS TRUE AS $f3]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -91,7 +91,7 @@ GROUP BY a ($1, 2))], $f3=[IS TRUE(<>($1, 5))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -102,7 +102,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RETRACT($ +- Exchange(distribution=[hash[a, $f4]]) +- Calc(select=[a, b, (b <> 2) IS TRUE AS $f2, (b <> 5) IS TRUE AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -122,7 +122,7 @@ GROUP BY a ($1, 2))], $f3=[IS TRUE(<>($1, 5))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -135,7 +135,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RET +- LocalGroupAggregate(groupBy=[a, $f4], partialFinalType=[PARTIAL], select=[a, $f4, COUNT(distinct$0 b) FILTER $f2 AS count$0, SUM(b) FILTER $f3 AS sum$1, SUM(b) FILTER $f2 AS sum$2, DISTINCT(b) AS distinct$0]) +- Calc(select=[a, b, (b <> 2) IS TRUE AS $f2, (b <> 5) IS TRUE AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -158,7 +158,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[FIRST_VALUE($1)], EXPR$2=[LAST_VALUE($1)], +- LogicalProject(b=[$1], c=[$2]) +- LogicalAggregate(group=[{0}], b=[COUNT(DISTINCT $1)], c=[MAX($1)]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -170,7 +170,7 @@ GroupAggregate(groupBy=[b], select=[b, FIRST_VALUE_RETRACT(c) AS EXPR$1, LAST_VA +- Exchange(distribution=[hash[a]], changelogMode=[I]) +- Calc(select=[a, b], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -193,7 +193,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[FIRST_VALUE($1)], EXPR$2=[LAST_VALUE($1)], +- LogicalProject(b=[$1], c=[$2]) +- LogicalAggregate(group=[{0}], b=[COUNT(DISTINCT $1)], c=[MAX($1)]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -206,7 +206,7 @@ GroupAggregate(groupBy=[b], select=[b, FIRST_VALUE_RETRACT(c) AS EXPR$1, LAST_VA +- LocalGroupAggregate(groupBy=[a], select=[a, COUNT(distinct$0 b) AS count$0, MAX(b) AS max$1, DISTINCT(b) AS distinct$0], changelogMode=[I]) +- Calc(select=[a, b], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -229,7 +229,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[FIRST_VALUE($1)], EXPR$2=[LAST_VALUE($1)], +- LogicalProject(b=[$1], c=[$2]) +- LogicalAggregate(group=[{0}], b=[COUNT(DISTINCT $1)], c=[MAX($1)]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -245,7 +245,7 @@ GroupAggregate(groupBy=[b], partialFinalType=[FINAL], select=[b, FIRST_VALUE_RET +- Exchange(distribution=[hash[a, $f2]], changelogMode=[I]) +- Calc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -268,7 +268,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[FIRST_VALUE($1)], EXPR$2=[LAST_VALUE($1)], +- LogicalProject(b=[$1], c=[$2]) +- LogicalAggregate(group=[{0}], b=[COUNT(DISTINCT $1)], c=[MAX($1)]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -286,7 +286,7 @@ GroupAggregate(groupBy=[b], partialFinalType=[FINAL], select=[b, FIRST_VALUE_RET +- LocalGroupAggregate(groupBy=[a, $f2], partialFinalType=[PARTIAL], select=[a, $f2, COUNT(distinct$0 b) AS count$0, MAX(b) AS max$1, DISTINCT(b) AS distinct$0], changelogMode=[I]) +- Calc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -298,7 +298,7 @@ GroupAggregate(groupBy=[b], partialFinalType=[FINAL], select=[b, FIRST_VALUE_RET @@ -307,7 +307,7 @@ GroupAggregate(groupBy=[a], select=[a, LISTAGG(DISTINCT c, $f2) AS EXPR$1]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, c, '#' AS $f2]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -319,7 +319,7 @@ GroupAggregate(groupBy=[a], select=[a, LISTAGG(DISTINCT c, $f2) AS EXPR$1]) @@ -329,7 +329,7 @@ GlobalGroupAggregate(groupBy=[a], select=[a, LISTAGG(distinct$0 (accDelimiter$0, +- LocalGroupAggregate(groupBy=[a], select=[a, LISTAGG(distinct$0 c, $f2) AS (accDelimiter$0, concatAcc$1), DISTINCT(c, $f2) AS distinct$0]) +- Calc(select=[a, c, '#' AS $f2]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -341,7 +341,7 @@ GlobalGroupAggregate(groupBy=[a], select=[a, LISTAGG(distinct$0 (accDelimiter$0, @@ -350,7 +350,7 @@ GroupAggregate(groupBy=[a], select=[a, LISTAGG(DISTINCT c, $f2) AS EXPR$1]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, c, '#' AS $f2]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -362,7 +362,7 @@ GroupAggregate(groupBy=[a], select=[a, LISTAGG(DISTINCT c, $f2) AS EXPR$1]) @@ -372,7 +372,7 @@ GlobalGroupAggregate(groupBy=[a], select=[a, LISTAGG(distinct$0 (accDelimiter$0, +- LocalGroupAggregate(groupBy=[a], select=[a, LISTAGG(distinct$0 c, $f2) AS (accDelimiter$0, concatAcc$1), DISTINCT(c, $f2) AS distinct$0]) +- Calc(select=[a, c, '#' AS $f2]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -394,7 +394,7 @@ FROM( LogicalAggregate(group=[{0}], EXPR$1=[MIN($1)], EXPR$2=[MAX($1)], EXPR$3=[SUM($1)], EXPR$4=[COUNT()], EXPR$5=[COUNT(DISTINCT $2)]) +- LogicalProject(c=[$2], b=[$1], a=[$0]) +- LogicalAggregate(group=[{0}], b=[AVG($1)], c=[MAX($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -404,7 +404,7 @@ GroupAggregate(groupBy=[c], select=[c, MIN_RETRACT(b) AS EXPR$1, MAX_RETRACT(b) +- GroupAggregate(groupBy=[a], select=[a, AVG(b) AS b, MAX(c) AS c], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[a]], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -426,7 +426,7 @@ FROM( LogicalAggregate(group=[{0}], EXPR$1=[MIN($1)], EXPR$2=[MAX($1)], EXPR$3=[SUM($1)], EXPR$4=[COUNT()], EXPR$5=[COUNT(DISTINCT $2)]) +- LogicalProject(c=[$2], b=[$1], a=[$0]) +- LogicalAggregate(group=[{0}], b=[AVG($1)], c=[MAX($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -438,7 +438,7 @@ GlobalGroupAggregate(groupBy=[c], select=[c, MIN_RETRACT(min$0) AS EXPR$1, MAX_R +- Exchange(distribution=[hash[a]], changelogMode=[I]) +- LocalGroupAggregate(groupBy=[a], select=[a, AVG(b) AS (sum$0, count$1), MAX(c) AS max$2], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -460,7 +460,7 @@ FROM( LogicalAggregate(group=[{0}], EXPR$1=[MIN($1)], EXPR$2=[MAX($1)], EXPR$3=[SUM($1)], EXPR$4=[COUNT()], EXPR$5=[COUNT(DISTINCT $2)]) +- LogicalProject(c=[$2], b=[$1], a=[$0]) +- LogicalAggregate(group=[{0}], b=[AVG($1)], c=[MAX($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -475,7 +475,7 @@ GroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, MIN_RETRACT($f3 +- GroupAggregate(groupBy=[a], select=[a, AVG(b) AS b, MAX(c) AS c], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[a]], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -497,7 +497,7 @@ FROM( LogicalAggregate(group=[{0}], EXPR$1=[MIN($1)], EXPR$2=[MAX($1)], EXPR$3=[SUM($1)], EXPR$4=[COUNT()], EXPR$5=[COUNT(DISTINCT $2)]) +- LogicalProject(c=[$2], b=[$1], a=[$0]) +- LogicalAggregate(group=[{0}], b=[AVG($1)], c=[MAX($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -515,7 +515,7 @@ GlobalGroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, MIN_RETRA +- Exchange(distribution=[hash[a]], changelogMode=[I]) +- LocalGroupAggregate(groupBy=[a], select=[a, AVG(b) AS (sum$0, count$1), MAX(c) AS max$2], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -527,7 +527,7 @@ GlobalGroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, MIN_RETRA @@ -536,7 +536,7 @@ GroupAggregate(select=[COUNT(DISTINCT a) AS EXPR$0, SUM(DISTINCT b) AS EXPR$1]) +- Exchange(distribution=[single]) +- Calc(select=[a, b]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -548,7 +548,7 @@ GroupAggregate(select=[COUNT(DISTINCT a) AS EXPR$0, SUM(DISTINCT b) AS EXPR$1]) @@ -558,7 +558,7 @@ GlobalGroupAggregate(select=[COUNT(distinct$0 count$0) AS EXPR$0, SUM(distinct$1 +- LocalGroupAggregate(select=[COUNT(distinct$0 a) AS count$0, SUM(distinct$1 b) AS sum$1, DISTINCT(a) AS distinct$0, DISTINCT(b) AS distinct$1]) +- Calc(select=[a, b]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -570,7 +570,7 @@ GlobalGroupAggregate(select=[COUNT(distinct$0 count$0) AS EXPR$0, SUM(distinct$1 @@ -583,7 +583,7 @@ GroupAggregate(partialFinalType=[FINAL], select=[$SUM0_RETRACT($f2_0) AS $f0, SU +- Expand(projects=[{a, b, $f2, null AS $f3, 1 AS $e}, {a, b, null AS $f2, $f3, 2 AS $e}]) +- Calc(select=[a, b, MOD(HASH_CODE(a), 1024) AS $f2, MOD(HASH_CODE(b), 1024) AS $f3]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -595,7 +595,7 @@ GroupAggregate(partialFinalType=[FINAL], select=[$SUM0_RETRACT($f2_0) AS $f0, SU @@ -610,7 +610,7 @@ GlobalGroupAggregate(partialFinalType=[FINAL], select=[$SUM0_RETRACT(sum$0) AS $ +- Expand(projects=[{a, b, $f2, null AS $f3, 1 AS $e}, {a, b, null AS $f2, $f3, 2 AS $e}]) +- Calc(select=[a, b, MOD(HASH_CODE(a), 1024) AS $f2, MOD(HASH_CODE(b), 1024) AS $f3]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -632,7 +632,7 @@ FROM( LogicalAggregate(group=[{0}], EXPR$1=[MIN($1)], EXPR$2=[MAX($1)], EXPR$3=[SUM($1)], EXPR$4=[COUNT()], EXPR$5=[COUNT(DISTINCT $2)]) +- LogicalProject(c=[$2], b=[$1], a=[$0]) +- LogicalAggregate(group=[{0}], b=[AVG($1)], c=[MAX($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -642,7 +642,7 @@ GroupAggregate(groupBy=[c], select=[c, MIN_RETRACT(b) AS EXPR$1, MAX_RETRACT(b) +- GroupAggregate(groupBy=[a], select=[a, AVG(b) AS b, MAX(c) AS c]) +- Exchange(distribution=[hash[a]]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -664,7 +664,7 @@ FROM( LogicalAggregate(group=[{0}], EXPR$1=[MIN($1)], EXPR$2=[MAX($1)], EXPR$3=[SUM($1)], EXPR$4=[COUNT()], EXPR$5=[COUNT(DISTINCT $2)]) +- LogicalProject(c=[$2], b=[$1], a=[$0]) +- LogicalAggregate(group=[{0}], b=[AVG($1)], c=[MAX($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -676,7 +676,7 @@ GlobalGroupAggregate(groupBy=[c], select=[c, MIN_RETRACT(min$0) AS EXPR$1, MAX_R +- Exchange(distribution=[hash[a]]) +- LocalGroupAggregate(groupBy=[a], select=[a, AVG(b) AS (sum$0, count$1), MAX(c) AS max$2]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -698,7 +698,7 @@ FROM( LogicalAggregate(group=[{0}], EXPR$1=[MIN($1)], EXPR$2=[MAX($1)], EXPR$3=[SUM($1)], EXPR$4=[COUNT()], EXPR$5=[COUNT(DISTINCT $2)]) +- LogicalProject(c=[$2], b=[$1], a=[$0]) +- LogicalAggregate(group=[{0}], b=[AVG($1)], c=[MAX($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -713,7 +713,7 @@ GroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, MIN_RETRACT($f3 +- GroupAggregate(groupBy=[a], select=[a, AVG(b) AS b, MAX(c) AS c]) +- Exchange(distribution=[hash[a]]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -735,7 +735,7 @@ FROM( LogicalAggregate(group=[{0}], EXPR$1=[MIN($1)], EXPR$2=[MAX($1)], EXPR$3=[SUM($1)], EXPR$4=[COUNT()], EXPR$5=[COUNT(DISTINCT $2)]) +- LogicalProject(c=[$2], b=[$1], a=[$0]) +- LogicalAggregate(group=[{0}], b=[AVG($1)], c=[MAX($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -753,7 +753,7 @@ GlobalGroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, MIN_RETRA +- Exchange(distribution=[hash[a]]) +- LocalGroupAggregate(groupBy=[a], select=[a, AVG(b) AS (sum$0, count$1), MAX(c) AS max$2]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -765,7 +765,7 @@ GlobalGroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, MIN_RETRA @@ -774,7 +774,7 @@ GroupAggregate(select=[COUNT(DISTINCT c) AS EXPR$0]) +- Exchange(distribution=[single]) +- Calc(select=[c]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -786,7 +786,7 @@ GroupAggregate(select=[COUNT(DISTINCT c) AS EXPR$0]) @@ -796,7 +796,7 @@ GlobalGroupAggregate(select=[COUNT(distinct$0 count$0) AS EXPR$0]) +- LocalGroupAggregate(select=[COUNT(distinct$0 c) AS count$0, DISTINCT(c) AS distinct$0]) +- Calc(select=[c]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -808,7 +808,7 @@ GlobalGroupAggregate(select=[COUNT(distinct$0 count$0) AS EXPR$0]) @@ -819,7 +819,7 @@ GroupAggregate(partialFinalType=[FINAL], select=[$SUM0_RETRACT($f1_0) AS $f0]) +- Exchange(distribution=[hash[$f1]]) +- Calc(select=[c, MOD(HASH_CODE(c), 1024) AS $f1]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -831,7 +831,7 @@ GroupAggregate(partialFinalType=[FINAL], select=[$SUM0_RETRACT($f1_0) AS $f0]) @@ -844,7 +844,7 @@ GlobalGroupAggregate(partialFinalType=[FINAL], select=[$SUM0_RETRACT(sum$0) AS $ +- LocalGroupAggregate(groupBy=[$f1], partialFinalType=[PARTIAL], select=[$f1, COUNT(distinct$0 c) AS count$0, DISTINCT(c) AS distinct$0]) +- Calc(select=[c, MOD(HASH_CODE(c), 1024) AS $f1]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -860,7 +860,7 @@ GROUP BY a @@ -869,7 +869,7 @@ Calc(select=[a, EXPR$1, CASE(($f3 = 0), null:BIGINT, EXPR$2) AS EXPR$2, (CASE(($ +- GroupAggregate(groupBy=[a], select=[a, COUNT(DISTINCT c) AS EXPR$1, $SUM0(b) AS EXPR$2, COUNT(b) AS $f3, MAX(b) AS EXPR$4, MIN(b) AS EXPR$5, COUNT(*) AS EXPR$7]) +- Exchange(distribution=[hash[a]]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -885,7 +885,7 @@ GROUP BY a @@ -895,7 +895,7 @@ Calc(select=[a, EXPR$1, CASE(($f3 = 0), null:BIGINT, EXPR$2) AS EXPR$2, (CASE(($ +- Exchange(distribution=[hash[a]]) +- LocalGroupAggregate(groupBy=[a], select=[a, COUNT(distinct$0 c) AS count$0, $SUM0(b) AS sum$1, COUNT(b) AS count$2, MAX(b) AS max$3, MIN(b) AS min$4, COUNT(*) AS count1$5, DISTINCT(c) AS distinct$0]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -911,7 +911,7 @@ GROUP BY a @@ -925,7 +925,7 @@ Calc(select=[a, $f1 AS EXPR$1, CASE(($f3 = 0), null:BIGINT, $f2) AS EXPR$2, (CAS +- Expand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}, {a, b, c, null AS $f3, null AS $f4, 3 AS $e}]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -941,7 +941,7 @@ GROUP BY a @@ -957,7 +957,7 @@ Calc(select=[a, $f1 AS EXPR$1, CASE(($f3 = 0), null:BIGINT, $f2) AS EXPR$2, (CAS +- Expand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}, {a, b, c, null AS $f3, null AS $f4, 3 AS $e}]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -969,7 +969,7 @@ Calc(select=[a, $f1 AS EXPR$1, CASE(($f3 = 0), null:BIGINT, $f2) AS EXPR$2, (CAS @@ -978,7 +978,7 @@ GroupAggregate(groupBy=[a], select=[a, COUNT(DISTINCT b) AS EXPR$1, SUM(b) AS EX +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -990,7 +990,7 @@ GroupAggregate(groupBy=[a], select=[a, COUNT(DISTINCT b) AS EXPR$1, SUM(b) AS EX @@ -1000,7 +1000,7 @@ GlobalGroupAggregate(groupBy=[a], select=[a, COUNT(distinct$0 count$0) AS EXPR$1 +- LocalGroupAggregate(groupBy=[a], select=[a, COUNT(distinct$0 b) AS count$0, SUM(b) AS sum$1, AVG(b) AS (sum$2, count$3), DISTINCT(b) AS distinct$0]) +- Calc(select=[a, b]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1012,7 +1012,7 @@ GlobalGroupAggregate(groupBy=[a], select=[a, COUNT(distinct$0 count$0) AS EXPR$1 @@ -1023,7 +1023,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RETRACT($ +- Exchange(distribution=[hash[a, $f2]]) +- Calc(select=[a, c, MOD(HASH_CODE(c), 1024) AS $f2]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1035,7 +1035,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RETRACT($ @@ -1047,7 +1047,7 @@ Calc(select=[a, $f1, $f2, IF(($f4 = 0), null:BIGINT, ($f3 / $f4)) AS $f3]) +- Exchange(distribution=[hash[a, $f2]]) +- Calc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1059,7 +1059,7 @@ Calc(select=[a, $f1, $f2, IF(($f4 = 0), null:BIGINT, ($f3 / $f4)) AS $f3]) @@ -1073,7 +1073,7 @@ Calc(select=[a, $f1, $f2, IF(($f4 = 0), null:BIGINT, ($f3 / $f4)) AS $f3]) +- LocalGroupAggregate(groupBy=[a, $f2], partialFinalType=[PARTIAL], select=[a, $f2, COUNT(distinct$0 b) AS count$0, SUM(b) AS sum$1, $SUM0(b) AS sum$2, COUNT(b) AS count$3, DISTINCT(b) AS distinct$0]) +- Calc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1085,7 +1085,7 @@ Calc(select=[a, $f1, $f2, IF(($f4 = 0), null:BIGINT, ($f3 / $f4)) AS $f3]) @@ -1094,7 +1094,7 @@ GroupAggregate(groupBy=[a], select=[a, COUNT(DISTINCT c) AS EXPR$1]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, c]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1106,7 +1106,7 @@ GroupAggregate(groupBy=[a], select=[a, COUNT(DISTINCT c) AS EXPR$1]) @@ -1116,7 +1116,7 @@ GlobalGroupAggregate(groupBy=[a], select=[a, COUNT(distinct$0 count$0) AS EXPR$1 +- LocalGroupAggregate(groupBy=[a], select=[a, COUNT(distinct$0 c) AS count$0, DISTINCT(c) AS distinct$0]) +- Calc(select=[a, c]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1128,7 +1128,7 @@ GlobalGroupAggregate(groupBy=[a], select=[a, COUNT(distinct$0 count$0) AS EXPR$1 @@ -1141,7 +1141,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RET +- LocalGroupAggregate(groupBy=[a, $f2], partialFinalType=[PARTIAL], select=[a, $f2, COUNT(distinct$0 c) AS count$0, DISTINCT(c) AS distinct$0]) +- Calc(select=[a, c, MOD(HASH_CODE(c), 1024) AS $f2]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1162,7 +1162,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[COUNT()]) +- LogicalProject(a=[$1], b=[$2]) +- LogicalAggregate(group=[{0}], a=[AVG($1)], b=[AVG($2)]) +- LogicalProject(c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1173,7 +1173,7 @@ GroupAggregate(groupBy=[a], select=[a, COUNT_RETRACT(DISTINCT b) AS EXPR$1, COUN +- GroupAggregate(groupBy=[c], select=[c, AVG(a) AS a, AVG(b) AS b], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[c]], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -1194,7 +1194,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[COUNT()]) +- LogicalProject(a=[$1], b=[$2]) +- LogicalAggregate(group=[{0}], a=[AVG($1)], b=[AVG($2)]) +- LogicalProject(c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1207,7 +1207,7 @@ GlobalGroupAggregate(groupBy=[a], select=[a, COUNT_RETRACT(distinct$0 count$0) A +- Exchange(distribution=[hash[c]], changelogMode=[I]) +- LocalGroupAggregate(groupBy=[c], select=[c, AVG(a) AS (sum$0, count$1), AVG(b) AS (sum$2, count$3)], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -1228,7 +1228,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[COUNT()]) +- LogicalProject(a=[$1], b=[$2]) +- LogicalAggregate(group=[{0}], a=[AVG($1)], b=[AVG($2)]) +- LogicalProject(c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1241,7 +1241,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RETRACT($ +- GroupAggregate(groupBy=[c], select=[c, AVG(a) AS a, AVG(b) AS b], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[c]], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -1262,7 +1262,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[COUNT()]) +- LogicalProject(a=[$1], b=[$2]) +- LogicalAggregate(group=[{0}], a=[AVG($1)], b=[AVG($2)]) +- LogicalProject(c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1278,7 +1278,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RET +- Exchange(distribution=[hash[c]], changelogMode=[I]) +- LocalGroupAggregate(groupBy=[c], select=[c, AVG(a) AS (sum$0, count$1), AVG(b) AS (sum$2, count$3)], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -1290,7 +1290,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RET @@ -1298,7 +1298,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[FIRST_VALUE($1)], EXPR$2=[COUNT(DISTINCT $ GroupAggregate(groupBy=[a], select=[a, FIRST_VALUE(c) AS EXPR$1, COUNT(DISTINCT b) AS EXPR$2]) +- Exchange(distribution=[hash[a]]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1310,7 +1310,7 @@ GroupAggregate(groupBy=[a], select=[a, FIRST_VALUE(c) AS EXPR$1, COUNT(DISTINCT @@ -1318,7 +1318,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[FIRST_VALUE($1)], EXPR$2=[COUNT(DISTINCT $ GroupAggregate(groupBy=[a], select=[a, FIRST_VALUE(c) AS EXPR$1, COUNT(DISTINCT b) AS EXPR$2]) +- Exchange(distribution=[hash[a]]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1330,7 +1330,7 @@ GroupAggregate(groupBy=[a], select=[a, FIRST_VALUE(c) AS EXPR$1, COUNT(DISTINCT @@ -1343,7 +1343,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, FIRST_VALUE_RET +- Expand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1355,7 +1355,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, FIRST_VALUE_RET @@ -1368,7 +1368,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, FIRST_VALUE_RET +- Expand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1380,7 +1380,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, FIRST_VALUE_RET @@ -1388,7 +1388,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[LAST_VALUE($1)], EXPR$2=[COUNT(DISTINCT $2 GroupAggregate(groupBy=[a], select=[a, LAST_VALUE(c) AS EXPR$1, COUNT(DISTINCT b) AS EXPR$2]) +- Exchange(distribution=[hash[a]]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1400,7 +1400,7 @@ GroupAggregate(groupBy=[a], select=[a, LAST_VALUE(c) AS EXPR$1, COUNT(DISTINCT b @@ -1408,7 +1408,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[LAST_VALUE($1)], EXPR$2=[COUNT(DISTINCT $2 GroupAggregate(groupBy=[a], select=[a, LAST_VALUE(c) AS EXPR$1, COUNT(DISTINCT b) AS EXPR$2]) +- Exchange(distribution=[hash[a]]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1420,7 +1420,7 @@ GroupAggregate(groupBy=[a], select=[a, LAST_VALUE(c) AS EXPR$1, COUNT(DISTINCT b @@ -1433,7 +1433,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, LAST_VALUE_RETR +- Expand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1445,7 +1445,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, LAST_VALUE_RETR @@ -1458,7 +1458,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, LAST_VALUE_RETR +- Expand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1470,7 +1470,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, LAST_VALUE_RETR @@ -1478,7 +1478,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[LISTAGG($1)], EXPR$2=[COUNT(DISTINCT $2)]) GroupAggregate(groupBy=[a], select=[a, LISTAGG(c) AS EXPR$1, COUNT(DISTINCT b) AS EXPR$2]) +- Exchange(distribution=[hash[a]]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1490,7 +1490,7 @@ GroupAggregate(groupBy=[a], select=[a, LISTAGG(c) AS EXPR$1, COUNT(DISTINCT b) A @@ -1499,7 +1499,7 @@ GlobalGroupAggregate(groupBy=[a], select=[a, LISTAGG((accDelimiter$0, concatAcc$ +- Exchange(distribution=[hash[a]]) +- LocalGroupAggregate(groupBy=[a], select=[a, LISTAGG(c) AS (accDelimiter$0, concatAcc$1), COUNT(distinct$0 b) AS count$2, DISTINCT(b) AS distinct$0]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1511,7 +1511,7 @@ GlobalGroupAggregate(groupBy=[a], select=[a, LISTAGG((accDelimiter$0, concatAcc$ @@ -1522,7 +1522,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, LISTAGG_RETRACT +- Exchange(distribution=[hash[a, $f3]]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1534,7 +1534,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, LISTAGG_RETRACT @@ -1547,7 +1547,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, LISTAGG_R +- LocalGroupAggregate(groupBy=[a, $f3], partialFinalType=[PARTIAL], select=[a, $f3, LISTAGG(c) AS (accDelimiter$0, concatAcc$1), COUNT(distinct$0 b) AS count$2, DISTINCT(b) AS distinct$0]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1562,7 +1562,7 @@ GROUP BY a @@ -1570,7 +1570,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[MAX($2)]) GroupAggregate(groupBy=[a], select=[a, COUNT(DISTINCT b) AS EXPR$1, MAX(c) AS EXPR$2]) +- Exchange(distribution=[hash[a]]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1585,7 +1585,7 @@ GROUP BY a @@ -1594,7 +1594,7 @@ GlobalGroupAggregate(groupBy=[a], select=[a, COUNT(distinct$0 count$0) AS EXPR$1 +- Exchange(distribution=[hash[a]]) +- LocalGroupAggregate(groupBy=[a], select=[a, COUNT(distinct$0 b) AS count$0, MAX(c) AS max$1, DISTINCT(b) AS distinct$0]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1609,7 +1609,7 @@ GROUP BY a @@ -1622,7 +1622,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RETRACT($ +- Expand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1637,7 +1637,7 @@ GROUP BY a @@ -1652,7 +1652,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RET +- Expand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1664,7 +1664,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RET @@ -1673,7 +1673,7 @@ GroupAggregate(groupBy=[a], select=[a, COUNT(DISTINCT a) AS EXPR$1, COUNT(b) AS +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1685,7 +1685,7 @@ GroupAggregate(groupBy=[a], select=[a, COUNT(DISTINCT a) AS EXPR$1, COUNT(b) AS @@ -1695,7 +1695,7 @@ GlobalGroupAggregate(groupBy=[a], select=[a, COUNT(distinct$0 count$0) AS EXPR$1 +- LocalGroupAggregate(groupBy=[a], select=[a, COUNT(distinct$0 a) AS count$0, COUNT(b) AS count$1, DISTINCT(a) AS distinct$0]) +- Calc(select=[a, b]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1718,7 +1718,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT(DISTINCT $1)], EXP +- LogicalProject(b=[$1], b1=[$2]) +- LogicalAggregate(group=[{0}], b=[COUNT($1)], b1=[MAX($1)]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1732,7 +1732,7 @@ GroupAggregate(groupBy=[b], partialFinalType=[FINAL], select=[b, SUM_RETRACT($f2 +- Exchange(distribution=[hash[a]], changelogMode=[I]) +- Calc(select=[a, b], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -1744,7 +1744,7 @@ GroupAggregate(groupBy=[b], partialFinalType=[FINAL], select=[b, SUM_RETRACT($f2 @@ -1755,7 +1755,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RETRACT($ +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1767,7 +1767,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RETRACT($ @@ -1780,7 +1780,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RET +- LocalGroupAggregate(groupBy=[a], partialFinalType=[PARTIAL], select=[a, COUNT(distinct$0 a) AS count$0, COUNT(b) AS count$1, DISTINCT(a) AS distinct$0]) +- Calc(select=[a, b]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1803,7 +1803,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT(DISTINCT $1)], EXP +- LogicalProject(b=[$1], b1=[$2]) +- LogicalAggregate(group=[{0}], b=[COUNT($1)], b1=[MAX($1)]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1815,7 +1815,7 @@ GroupAggregate(groupBy=[b], select=[b, SUM_RETRACT(b1) AS EXPR$1, COUNT_RETRACT( +- Exchange(distribution=[hash[a]], changelogMode=[I]) +- Calc(select=[a, b], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -1838,7 +1838,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT(DISTINCT $1)], EXP +- LogicalProject(b=[$1], b1=[$2]) +- LogicalAggregate(group=[{0}], b=[COUNT($1)], b1=[MAX($1)]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1852,7 +1852,7 @@ GlobalGroupAggregate(groupBy=[b], select=[b, SUM_RETRACT((sum$0, count$1)) AS EX +- LocalGroupAggregate(groupBy=[a], select=[a, COUNT(b) AS count$0, MAX(b) AS max$1], changelogMode=[I]) +- Calc(select=[a, b], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -1864,7 +1864,7 @@ GlobalGroupAggregate(groupBy=[b], select=[b, SUM_RETRACT((sum$0, count$1)) AS EX @@ -1877,7 +1877,7 @@ GroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, SUM_RETRACT($f3 +- Expand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}, {a, b, c, null AS $f3, null AS $f4, 3 AS $e}]) +- Calc(select=[a, b, c, MOD(HASH_CODE(a), 1024) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1900,7 +1900,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT(DISTINCT $1)], EXP +- LogicalProject(b=[$1], b1=[$2]) +- LogicalAggregate(group=[{0}], b=[COUNT($1)], b1=[MAX($1)]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1917,7 +1917,7 @@ GlobalGroupAggregate(groupBy=[b], partialFinalType=[FINAL], select=[b, SUM_RETRA +- LocalGroupAggregate(groupBy=[a], select=[a, COUNT(b) AS count$0, MAX(b) AS max$1], changelogMode=[I]) +- Calc(select=[a, b], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -1929,7 +1929,7 @@ GlobalGroupAggregate(groupBy=[b], partialFinalType=[FINAL], select=[b, SUM_RETRA @@ -1938,7 +1938,7 @@ GlobalGroupAggregate(groupBy=[c], select=[c, SUM(distinct$0 sum$0) AS EXPR$1, SU +- Exchange(distribution=[hash[c]]) +- LocalGroupAggregate(groupBy=[c], select=[c, SUM(distinct$0 a) AS sum$0, SUM(a) AS sum$1, COUNT(distinct$1 b) AS count$2, DISTINCT(a) AS distinct$0, DISTINCT(b) AS distinct$1]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1950,7 +1950,7 @@ GlobalGroupAggregate(groupBy=[c], select=[c, SUM(distinct$0 sum$0) AS EXPR$1, SU @@ -1965,7 +1965,7 @@ GlobalGroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, SUM_RETRA +- Expand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}, {a, b, c, null AS $f3, null AS $f4, 3 AS $e}]) +- Calc(select=[a, b, c, MOD(HASH_CODE(a), 1024) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -1977,7 +1977,7 @@ GlobalGroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, SUM_RETRA @@ -1985,7 +1985,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM(DISTINCT $1)], EXPR$2=[SUM($1)], EXPR$ GroupAggregate(groupBy=[c], select=[c, SUM(DISTINCT a) AS EXPR$1, SUM(a) AS EXPR$2, COUNT(DISTINCT b) AS EXPR$3]) +- Exchange(distribution=[hash[c]]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/GroupingSetsTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/GroupingSetsTest.xml index 547debd1d3017..886793c329129 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/GroupingSetsTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/GroupingSetsTest.xml @@ -28,7 +28,7 @@ FROM emp GROUP BY CUBE(MOD(deptno, 20), gender) LogicalProject(d=[$0], c=[$2], g=[$1]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], c=[COUNT()]) +- LogicalProject(d=[MOD($1, 20)], g=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -38,7 +38,7 @@ Calc(select=[d, c, g]) +- Exchange(distribution=[hash[d, g, $e]]) +- Expand(projects=[{d, g, 0 AS $e}, {d, null AS g, 1 AS $e}, {null AS d, g, 2 AS $e}, {null AS d, null AS g, 3 AS $e}]) +- Calc(select=[MOD(deptno, 20) AS d, gender AS g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -51,7 +51,7 @@ Calc(select=[d, c, g]) LogicalProject(c=[$1]) +- LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()]) +- LogicalProject($f0=[1]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -61,7 +61,7 @@ Calc(select=[c]) +- Exchange(distribution=[hash[$f0, $e]]) +- Expand(projects=[{$f0, 0 AS $e}, {null AS $f0, 1 AS $e}]) +- Calc(select=[1 AS $f0]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -77,7 +77,7 @@ FROM emp GROUP BY ROLLUP (deptno + 1) LogicalProject(d1=[$0], d0=[-($0, 1)], c=[$1]) +- LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()]) +- LogicalProject(d1=[+($1, 1)]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -87,7 +87,7 @@ Calc(select=[d1, (d1 - 1) AS d0, c]) +- Exchange(distribution=[hash[d1, $e]]) +- Expand(projects=[{d1, 0 AS $e}, {null AS d1, 1 AS $e}]) +- Calc(select=[(deptno + 1) AS d1]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -103,7 +103,7 @@ FROM emp GROUP BY ROLLUP(MOD(deptno, 20), gender) LogicalProject(d=[$0], c=[$2], g=[$1]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], c=[COUNT()]) +- LogicalProject(d=[MOD($1, 20)], g=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -113,7 +113,7 @@ Calc(select=[d, c, g]) +- Exchange(distribution=[hash[d, g, $e]]) +- Expand(projects=[{d, g, 0 AS $e}, {d, null AS g, 1 AS $e}, {null AS d, null AS g, 3 AS $e}]) +- Calc(select=[MOD(deptno, 20) AS d, gender AS g]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -126,7 +126,7 @@ Calc(select=[d, c, g]) LogicalProject(c=[$1]) +- LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()]) +- LogicalProject($f0=[1]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -136,7 +136,7 @@ Calc(select=[c]) +- Exchange(distribution=[hash[$f0, $e]]) +- Expand(projects=[{$f0, 0 AS $e}, {null AS $f0, 1 AS $e}]) +- Calc(select=[1 AS $f0]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -148,7 +148,7 @@ Calc(select=[c]) @@ -158,7 +158,7 @@ Calc(select=[deptno, c]) +- Exchange(distribution=[hash[deptno, $e]]) +- Expand(projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) +- Calc(select=[deptno]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emps, source: [TestTableSource(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)]]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat]) + +- TableSourceScan(table=[[default_catalog, default_database, emps]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat]) ]]> @@ -182,7 +182,7 @@ FROM MyTable LogicalProject(b=[$0], c=[$1], a=[$2], g=[0:BIGINT], gb=[$3], gc=[$4], gib=[$5], gic=[$6], gid=[$7]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], a=[AVG($2)], gb=[GROUPING($0)], gc=[GROUPING($1)], gib=[GROUPING_ID($0)], gic=[GROUPING_ID($1)], gid=[GROUPING_ID($0, $1)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -191,7 +191,7 @@ Calc(select=[b, c, a, 0 AS g, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS gb, CASE(($e +- GroupAggregate(groupBy=[b, c, $e], select=[b, c, $e, AVG(a) AS a]) +- Exchange(distribution=[hash[b, c, $e]]) +- Expand(projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -205,7 +205,7 @@ LogicalAggregate(group=[{0}]) +- LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], EXPR$0=[COUNT()]) +- LogicalProject(deptno=[$1], gender=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -217,7 +217,7 @@ GroupAggregate(groupBy=[EXPR$0], select=[EXPR$0]) +- Exchange(distribution=[hash[deptno, gender, $e]]) +- Expand(projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -238,8 +238,8 @@ LogicalProject(deptno=[$0], gender=[$2], min_name=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0, 2}, {0}, {1, 2}, {1}, {2}, {}]], min_name=[MIN($3)], agg#1=[COUNT()]) +- LogicalProject(deptno=[$1], deptno0=[$3], gender=[$2], ename=[$0]) +- LogicalJoin(condition=[=($1, $3)], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, dept, source: [TestTableSource(deptno, dname)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, emp]]) + +- LogicalTableScan(table=[[default_catalog, default_database, dept]]) ]]> @@ -250,10 +250,10 @@ Calc(select=[deptno, gender, min_name], where=[(($f5 > 2) OR ((gender = 'M') AND +- Expand(projects=[{ename, deptno, gender, deptno0, 0 AS $e}, {ename, deptno, gender, null AS deptno0, 1 AS $e}, {ename, deptno, null AS gender, deptno0, 2 AS $e}, {ename, deptno, null AS gender, null AS deptno0, 3 AS $e}, {ename, null AS deptno, gender, deptno0, 4 AS $e}, {ename, null AS deptno, gender, null AS deptno0, 5 AS $e}, {ename, null AS deptno, null AS gender, deptno0, 6 AS $e}, {ename, null AS deptno, null AS gender, null AS deptno0, 7 AS $e}]) +- Join(joinType=[InnerJoin], where=[(deptno = deptno0)], select=[ename, deptno, gender, deptno0], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[deptno]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + : +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) +- Exchange(distribution=[hash[deptno]]) +- Calc(select=[deptno]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, dept, source: [TestTableSource(deptno, dname)]]], fields=[deptno, dname]) + +- TableSourceScan(table=[[default_catalog, default_database, dept]], fields=[deptno, dname]) ]]> @@ -277,7 +277,7 @@ FROM MyTable LogicalProject(b=[$0], c=[$1], a=[$2], g=[0:BIGINT], gb=[$3], gc=[$4], gib=[$5], gic=[$6], gid=[$7]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], a=[AVG($2)], gb=[GROUPING($0)], gc=[GROUPING($1)], gib=[GROUPING_ID($0)], gic=[GROUPING_ID($1)], gid=[GROUPING_ID($0, $1)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -286,7 +286,7 @@ Calc(select=[b, c, a, 0 AS g, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS gb, CASE(($e +- GroupAggregate(groupBy=[b, c, $e], select=[b, c, $e, AVG(a) AS a]) +- Exchange(distribution=[hash[b, c, $e]]) +- Expand(projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -306,7 +306,7 @@ LogicalProject(deptno=[$1], gender=[$0], EXPR$2=[$2], c=[$3]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], EXPR$2=[GROUPING_ID($1, $0, $1)], c=[COUNT()]) +- LogicalProject(gender=[$2], deptno=[$1]) +- LogicalFilter(condition=[=($1, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -316,7 +316,7 @@ Calc(select=[deptno, gender, CASE(($e = 0), 0, ($e = 1), 5, 7) AS EXPR$2, c]) +- Exchange(distribution=[hash[gender, deptno, $e]]) +- Expand(projects=[{gender, deptno, 0 AS $e}, {gender, null AS deptno, 1 AS $e}, {null AS gender, null AS deptno, 3 AS $e}]) +- Calc(select=[gender, CAST(10 AS INTEGER) AS deptno], where=[(deptno = 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -329,7 +329,7 @@ Calc(select=[deptno, gender, CASE(($e = 0), 0, ($e = 1), 5, 7) AS EXPR$2, c]) LogicalProject(EXPR$0=[+($0, 1)], c=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], c=[COUNT()]) +- LogicalProject(deptno=[$1], gender=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -339,7 +339,7 @@ Calc(select=[(deptno + 1) AS EXPR$0, c]) +- Exchange(distribution=[hash[deptno, gender, $e]]) +- Expand(projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -364,7 +364,7 @@ LogicalSort(sort0=[$0], sort1=[$1], sort2=[$2], dir0=[ASC-nulls-first], dir1=[AS +- LogicalProject(deptno=[$0], job=[$1], empno=[$2], ename=[$3], sumsal=[$4], gr_text=[CASE(=($5, 0), _UTF-16LE'grouped by deptno,job,empno,ename':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", =($5, 1), _UTF-16LE'grouped by deptno,job':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", =($5, 3), _UTF-16LE'grouped by deptno':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", =($5, 7), _UTF-16LE'grouped by ()':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", null:VARCHAR(2147483647) CHARACTER SET "UTF-16LE")]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 1}, {0}, {}]], sumsal=[SUM($4)], agg#1=[GROUPING_ID($0, $1, $2)]) +- LogicalProject(deptno=[$7], job=[$2], empno=[$0], ename=[$1], sal=[$5]) - +- LogicalTableScan(table=[[default_catalog, default_database, scott_emp, source: [TestTableSource(empno, ename, job, mgr, hiredate, sal, comm, deptno)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, scott_emp]]) ]]> @@ -376,7 +376,7 @@ Sort(orderBy=[deptno ASC, job ASC, empno ASC]) +- Exchange(distribution=[hash[deptno, job, empno, ename, $e]]) +- Expand(projects=[{deptno, job, empno, ename, sal, 0 AS $e}, {deptno, job, null AS empno, null AS ename, sal, 3 AS $e}, {deptno, null AS job, null AS empno, null AS ename, sal, 7 AS $e}, {null AS deptno, null AS job, null AS empno, null AS ename, sal, 15 AS $e}]) +- Calc(select=[deptno, job, empno, ename, sal]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, scott_emp, source: [TestTableSource(empno, ename, job, mgr, hiredate, sal, comm, deptno)]]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno]) + +- TableSourceScan(table=[[default_catalog, default_database, scott_emp]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno]) ]]> @@ -399,7 +399,7 @@ LogicalProject(c=[$2], gd=[$3], gid=[$4], gdgd=[$5], gidgd=[$6]) +- LogicalFilter(condition=[<=($3, $6)]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], c=[COUNT()], gd=[GROUPING($0)], gid=[GROUPING_ID($0)], gdgd=[GROUPING($0, $1, $0)], gidgd=[GROUPING_ID($0, $1, $0)]) +- LogicalProject(deptno=[$1], gender=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -410,7 +410,7 @@ Calc(select=[c, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS gd, CASE(SEARCH($e, Sarg[0 +- Calc(select=[deptno, gender, $e], where=[(CASE(SEARCH($e, Sarg[0, 1]), 0, 1) <= CASE(($e = 0), 0, ($e = 1), 2, 7))]) +- Expand(projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -433,7 +433,7 @@ LogicalProject(c=[$2], g=[$3], gid=[$4], gd=[$5], gg=[$6], ggd=[$7], gdg=[$8]) +- LogicalProject(deptno=[$0], gender=[$1], c=[$2], g=[$3], gid=[0:BIGINT], gd=[$4], gg=[$5], ggd=[$6], gdg=[$7]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], c=[COUNT()], g=[GROUPING($0)], gd=[GROUPING_ID($0)], gg=[GROUPING_ID($1)], ggd=[GROUPING_ID($1, $0)], gdg=[GROUPING_ID($0, $1)]) +- LogicalProject(deptno=[$1], gender=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -443,7 +443,7 @@ Calc(select=[c, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS g, 0 AS gid, CASE(SEARCH($ +- Exchange(distribution=[hash[deptno, gender, $e]]) +- Expand(projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -467,7 +467,7 @@ LogicalProject(deptno=[$0], gender=[$1], gd=[$2], gg=[$3], dg=[$4], gd0=[$5], gi +- LogicalProject(deptno=[$0], gender=[$1], gd=[$2], gg=[$3], dg=[$4], gd_0=[$5], gid=[0:BIGINT], c=[$6]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], gd=[GROUPING($0)], gg=[GROUPING($1)], dg=[GROUPING_ID($0, $1)], gd=[GROUPING_ID($1, $0)], c=[COUNT()]) +- LogicalProject(deptno=[$1], gender=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -477,7 +477,7 @@ Calc(select=[deptno, gender, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS gd, CASE(($e +- Exchange(distribution=[hash[deptno, gender, $e]]) +- Expand(projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -497,7 +497,7 @@ FROM scott_emp GROUP BY CUBE(deptno, job) @@ -507,7 +507,7 @@ Calc(select=[deptno, job, c, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS d, CASE(($e = +- Exchange(distribution=[hash[deptno, job, $e]]) +- Expand(projects=[{deptno, job, 0 AS $e}, {deptno, null AS job, 1 AS $e}, {null AS deptno, job, 2 AS $e}, {null AS deptno, null AS job, 3 AS $e}]) +- Calc(select=[deptno, job]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, scott_emp, source: [TestTableSource(empno, ename, job, mgr, hiredate, sal, comm, deptno)]]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno]) + +- TableSourceScan(table=[[default_catalog, default_database, scott_emp]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno]) ]]> @@ -520,7 +520,7 @@ Calc(select=[deptno, job, c, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS d, CASE(($e = LogicalProject(c=[$1], g=[$2]) +- LogicalAggregate(group=[{0}], c=[COUNT()], g=[GROUPING($0)]) +- LogicalProject(deptno=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -529,7 +529,7 @@ Calc(select=[c, 0 AS g]) +- GroupAggregate(groupBy=[deptno], select=[deptno, COUNT(*) AS c]) +- Exchange(distribution=[hash[deptno]]) +- Calc(select=[deptno]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -544,7 +544,7 @@ SELECT COUNT(*) AS c, deptno, GROUPING(deptno) AS g FROM emp GROUP BY ROLLUP(dep LogicalProject(c=[$1], deptno=[$0], g=[$2]) +- LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()], g=[GROUPING($0)]) +- LogicalProject(deptno=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -554,7 +554,7 @@ Calc(select=[c, deptno, CASE(($e = 0), 0, 1) AS g]) +- Exchange(distribution=[hash[deptno, $e]]) +- Expand(projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) +- Calc(select=[deptno]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -570,7 +570,7 @@ GROUP BY GROUPING SETS (b, c) LogicalProject(b=[$0], c=[$1], a=[$2], g=[0:BIGINT]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}]], a=[AVG($2)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -579,7 +579,7 @@ Calc(select=[b, c, a, 0 AS g]) +- GroupAggregate(groupBy=[b, c, $e], select=[b, c, $e, AVG(a) AS a]) +- Exchange(distribution=[hash[b, c, $e]]) +- Expand(projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -591,7 +591,7 @@ Calc(select=[b, c, a, 0 AS g]) @@ -600,7 +600,7 @@ Calc(select=[b, c, a]) +- GroupAggregate(groupBy=[b, c, $e], select=[b, c, $e, AVG(a) AS a]) +- Exchange(distribution=[hash[b, c, $e]]) +- Expand(projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -625,7 +625,7 @@ FROM MyTable LogicalProject(b=[$0], c=[$1], a=[$2], g=[0:BIGINT], gb=[$3], gc=[$4], gib=[$5], gic=[$6], gid=[$7], cnt=[$8]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}, {}]], a=[AVG($2)], gb=[GROUPING($0)], gc=[GROUPING($1)], gib=[GROUPING_ID($0)], gic=[GROUPING_ID($1)], gid=[GROUPING_ID($0, $1)], cnt=[COUNT()]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -634,7 +634,7 @@ Calc(select=[b, c, a, 0 AS g, CASE(($e = 1), 0, 1) AS gb, CASE(($e = 1), 1, ($e +- GroupAggregate(groupBy=[b, c, $e], select=[b, c, $e, AVG(a) AS a, COUNT(*) AS cnt]) +- Exchange(distribution=[hash[b, c, $e]]) +- Expand(projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -648,7 +648,7 @@ SELECT deptno + 1, COUNT(*) AS c FROM emps GROUP BY GROUPING SETS ((), (deptno + @@ -658,7 +658,7 @@ Calc(select=[EXPR$0, c]) +- Exchange(distribution=[hash[EXPR$0, $e]]) +- Expand(projects=[{EXPR$0, 0 AS $e}, {null AS EXPR$0, 1 AS $e}]) +- Calc(select=[(deptno + 1) AS EXPR$0]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emps, source: [TestTableSource(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)]]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat]) + +- TableSourceScan(table=[[default_catalog, default_database, emps]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat]) ]]> @@ -675,7 +675,7 @@ LogicalProject(c=[$0]) +- LogicalProject(c=[$1], EXPR$1=[$2]) +- LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()], agg#1=[GROUPING($0)]) +- LogicalProject(deptno=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -688,7 +688,7 @@ Calc(select=[c]) +- Exchange(distribution=[hash[deptno, $e]]) +- Expand(projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) +- Calc(select=[deptno]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -737,7 +737,7 @@ FROM emps GROUP BY GROUPING SETS (deptno) LogicalProject(deptno=[$0], a=[$1], g=[0:BIGINT], gb=[$2], gib=[$3]) +- LogicalAggregate(group=[{0}], a=[AVG($1)], gb=[GROUPING($0)], gib=[GROUPING_ID($0)]) +- LogicalProject(deptno=[$2], age=[$6]) - +- LogicalTableScan(table=[[default_catalog, default_database, emps, source: [TestTableSource(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emps]]) ]]> @@ -746,7 +746,7 @@ Calc(select=[deptno, a, 0 AS g, 0 AS gb, 0 AS gib]) +- GroupAggregate(groupBy=[deptno], select=[deptno, AVG(age) AS a]) +- Exchange(distribution=[hash[deptno]]) +- Calc(select=[deptno, age]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emps, source: [TestTableSource(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)]]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat]) + +- TableSourceScan(table=[[default_catalog, default_database, emps]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat]) ]]> @@ -770,7 +770,7 @@ FROM MyTable LogicalProject(b=[$0], c=[$1], a=[$2], g=[0:BIGINT], gb=[$3], gc=[$4], gib=[$5], gic=[$6], gid=[$7]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], a=[AVG($2)], gb=[GROUPING($0)], gc=[GROUPING($1)], gib=[GROUPING_ID($0)], gic=[GROUPING_ID($1)], gid=[GROUPING_ID($0, $1)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -779,7 +779,7 @@ Calc(select=[b, c, a, 0 AS g, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS gb, CASE(($e +- GroupAggregate(groupBy=[b, c, $e], select=[b, c, $e, AVG(a) AS a]) +- Exchange(distribution=[hash[b, c, $e]]) +- Expand(projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -803,7 +803,7 @@ FROM MyTable LogicalProject(b=[$0], c=[$1], a=[$2], g=[0:BIGINT], gb=[$3], gc=[$4], gib=[$5], gic=[$6], gid=[$7]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], a=[AVG($2)], gb=[GROUPING($0)], gc=[GROUPING($1)], gib=[GROUPING_ID($0)], gic=[GROUPING_ID($1)], gid=[GROUPING_ID($0, $1)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -812,7 +812,7 @@ Calc(select=[b, c, a, 0 AS g, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS gb, CASE(($e +- GroupAggregate(groupBy=[b, c, $e], select=[b, c, $e, AVG(a) AS a]) +- Exchange(distribution=[hash[b, c, $e]]) +- Expand(projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -825,7 +825,7 @@ Calc(select=[b, c, a, 0 AS g, CASE(SEARCH($e, Sarg[0, 1]), 0, 1) AS gb, CASE(($e LogicalProject(deptno=[$0], c=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], c=[COUNT()]) +- LogicalProject(deptno=[$1], gender=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -835,7 +835,7 @@ Calc(select=[deptno, c]) +- Exchange(distribution=[hash[deptno, gender, $e]]) +- Expand(projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -851,7 +851,7 @@ GROUP BY ROLLUP(deptno / 2, gender), ROLLUP(substring(ename FROM 1 FOR 1)) LogicalProject(half1=[+($0, 1)], c=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0, 2}, {0}, {2}, {}]], c=[COUNT()]) +- LogicalProject($f0=[/($1, 2)], gender=[$2], $f2=[SUBSTRING($0, 1, 1)]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -861,7 +861,7 @@ Calc(select=[($f0 + 1) AS half1, c]) +- Exchange(distribution=[hash[$f0, gender, $f2, $e]]) +- Expand(projects=[{$f0, gender, $f2, 0 AS $e}, {$f0, gender, null AS $f2, 1 AS $e}, {$f0, null AS gender, $f2, 2 AS $e}, {$f0, null AS gender, null AS $f2, 3 AS $e}, {null AS $f0, null AS gender, $f2, 6 AS $e}, {null AS $f0, null AS gender, null AS $f2, 7 AS $e}]) +- Calc(select=[(deptno / 2) AS $f0, gender, SUBSTRING(ename, 1, 1) AS $f2]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -876,7 +876,7 @@ SELECT gender, deptno + 1, COUNT(*) AS c FROM emp GROUP BY ROLLUP(deptno, gender LogicalProject(gender=[$1], EXPR$1=[+($0, 1)], c=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], c=[COUNT()]) +- LogicalProject(deptno=[$1], gender=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -886,7 +886,7 @@ Calc(select=[gender, (deptno + 1) AS EXPR$1, c]) +- Exchange(distribution=[hash[deptno, gender, $e]]) +- Expand(projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -898,7 +898,7 @@ Calc(select=[gender, (deptno + 1) AS EXPR$1, c]) @@ -908,7 +908,7 @@ Calc(select=[gender, c]) +- Exchange(distribution=[hash[gender, $e]]) +- Expand(projects=[{gender, 0 AS $e}, {null AS gender, 1 AS $e}]) +- Calc(select=[gender]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -921,7 +921,7 @@ Calc(select=[gender, c]) LogicalSort(sort0=[$1], dir0=[DESC-nulls-last]) +- LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()]) +- LogicalProject(gender=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -933,7 +933,7 @@ Sort(orderBy=[c DESC]) +- Exchange(distribution=[hash[gender, $e]]) +- Expand(projects=[{gender, 0 AS $e}, {null AS gender, 1 AS $e}]) +- Calc(select=[gender]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -949,7 +949,7 @@ LogicalProject(d1=[+($0, 1)], c=[$1]) +- LogicalFilter(condition=[>($1, 3)]) +- LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()]) +- LogicalProject(deptno=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -959,7 +959,7 @@ Calc(select=[(deptno + 1) AS d1, c], where=[(c > 3)]) +- Exchange(distribution=[hash[deptno, $e]]) +- Expand(projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) +- Calc(select=[deptno]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -972,7 +972,7 @@ Calc(select=[(deptno + 1) AS d1, c], where=[(c > 3)]) LogicalProject(EXPR$0=[+($0, 1)], c=[$1]) +- LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()]) +- LogicalProject(deptno=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, emp]]) ]]> @@ -982,7 +982,7 @@ Calc(select=[(deptno + 1) AS EXPR$0, c]) +- Exchange(distribution=[hash[deptno, $e]]) +- Expand(projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) +- Calc(select=[deptno]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- TableSourceScan(table=[[default_catalog, default_database, emp]], fields=[ename, deptno, gender]) ]]> @@ -1006,7 +1006,7 @@ LogicalProject(gt=[CASE(=($0, 1), _UTF-16LE'aaa', =($2, 1), _UTF-16LE'bbb', null +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1}, {1, 2, 3}]], a=[AVG($4)]) +- LogicalProject(g1=[$3], b=[$1], g2=[$4], c=[$2], a=[$0]) +- LogicalProject(a=[$0], b=[$1], c=[$2], g1=[1], g2=[1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -1016,7 +1016,7 @@ Calc(select=[CASE((g1 = 1), 'aaa', (g2 = 1), 'bbb', null:CHAR(3)) AS gt, b, c, a +- Exchange(distribution=[hash[g1, b, g2, c, $e]]) +- Expand(projects=[{g1, b, null AS g2, null AS c, a, 3 AS $e}, {null AS g1, b, g2, c, a, 8 AS $e}]) +- Calc(select=[1 AS g1, b, 1 AS g2, c, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.xml index 8585541ae0cf9..e054111339611 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.xml @@ -32,7 +32,7 @@ GROUP BY a ($1, 2))], $f3=[IS TRUE(<>($1, 5))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -44,7 +44,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0(cou +- LocalGroupAggregate(groupBy=[a, $f4], partialFinalType=[PARTIAL], select=[a, $f4, COUNT(distinct$0 b) FILTER $f2 AS count$0, SUM(b) FILTER $f3 AS sum$1, SUM(b) FILTER $f2 AS sum$2, DISTINCT(b) AS distinct$0]) +- Calc(select=[a, b, (b <> 2) IS TRUE AS $f2, (b <> 5) IS TRUE AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -67,7 +67,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[FIRST_VALUE($1)], EXPR$2=[LAST_VALUE($1)], +- LogicalProject(b=[$1], c=[$2]) +- LogicalAggregate(group=[{0}], b=[COUNT(DISTINCT $1)], c=[MAX($1)]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -84,7 +84,7 @@ GroupAggregate(groupBy=[b], partialFinalType=[FINAL], select=[b, FIRST_VALUE_RET +- LocalGroupAggregate(groupBy=[a, $f2], partialFinalType=[PARTIAL], select=[a, $f2, COUNT(distinct$0 b) AS count$0, MAX(b) AS max$1, DISTINCT(b) AS distinct$0], changelogMode=[I]) +- Calc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -96,7 +96,7 @@ GroupAggregate(groupBy=[b], partialFinalType=[FINAL], select=[b, FIRST_VALUE_RET @@ -106,7 +106,7 @@ GlobalGroupAggregate(groupBy=[a], select=[a, LISTAGG(distinct$0 (accDelimiter$0, +- LocalGroupAggregate(groupBy=[a], select=[a, LISTAGG(distinct$0 c, $f2) AS (accDelimiter$0, concatAcc$1), DISTINCT(c, $f2) AS distinct$0]) +- Calc(select=[a, c, '#' AS $f2]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -128,7 +128,7 @@ FROM( LogicalAggregate(group=[{0}], EXPR$1=[MIN($1)], EXPR$2=[MAX($1)], EXPR$3=[SUM($1)], EXPR$4=[COUNT()], EXPR$5=[COUNT(DISTINCT $2)]) +- LogicalProject(c=[$2], b=[$1], a=[$0]) +- LogicalAggregate(group=[{0}], b=[AVG($1)], c=[MAX($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -145,7 +145,7 @@ GlobalGroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, MIN_RETRA +- Exchange(distribution=[hash[a]], changelogMode=[I]) +- LocalGroupAggregate(groupBy=[a], select=[a, AVG(b) AS (sum$0, count$1), MAX(c) AS max$2], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -157,7 +157,7 @@ GlobalGroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, MIN_RETRA @@ -171,7 +171,7 @@ GlobalGroupAggregate(partialFinalType=[FINAL], select=[$SUM0(count$0) AS $f0, SU +- Expand(projects=[{a, b, $f2, null AS $f3, 1 AS $e}, {a, b, null AS $f2, $f3, 2 AS $e}]) +- Calc(select=[a, b, MOD(HASH_CODE(a), 1024) AS $f2, MOD(HASH_CODE(b), 1024) AS $f3]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -193,7 +193,7 @@ FROM( LogicalAggregate(group=[{0}], EXPR$1=[MIN($1)], EXPR$2=[MAX($1)], EXPR$3=[SUM($1)], EXPR$4=[COUNT()], EXPR$5=[COUNT(DISTINCT $2)]) +- LogicalProject(c=[$2], b=[$1], a=[$0]) +- LogicalAggregate(group=[{0}], b=[AVG($1)], c=[MAX($2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -210,7 +210,7 @@ GlobalGroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, MIN_RETRA +- Exchange(distribution=[hash[a]]) +- LocalGroupAggregate(groupBy=[a], select=[a, AVG(b) AS (sum$0, count$1), MAX(c) AS max$2]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -222,7 +222,7 @@ GlobalGroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, MIN_RETRA @@ -234,7 +234,7 @@ GlobalGroupAggregate(partialFinalType=[FINAL], select=[$SUM0(count$0) AS $f0]) +- LocalGroupAggregate(groupBy=[$f1], partialFinalType=[PARTIAL], select=[$f1, COUNT(distinct$0 c) AS count$0, DISTINCT(c) AS distinct$0]) +- Calc(select=[c, MOD(HASH_CODE(c), 1024) AS $f1]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -250,7 +250,7 @@ GROUP BY a @@ -265,7 +265,7 @@ Calc(select=[a, $f1 AS EXPR$1, CASE(($f3 = 0), null:BIGINT, $f2) AS EXPR$2, (CAS +- Expand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}, {a, b, c, null AS $f3, null AS $f4, 3 AS $e}]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -277,7 +277,7 @@ Calc(select=[a, $f1 AS EXPR$1, CASE(($f3 = 0), null:BIGINT, $f2) AS EXPR$2, (CAS @@ -289,7 +289,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0(cou +- LocalGroupAggregate(groupBy=[a, $f2], partialFinalType=[PARTIAL], select=[a, $f2, COUNT(distinct$0 c) AS count$0, DISTINCT(c) AS distinct$0]) +- Calc(select=[a, c, MOD(HASH_CODE(c), 1024) AS $f2]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -301,7 +301,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0(cou @@ -314,7 +314,7 @@ Calc(select=[a, $f1, $f2, IF(($f4 = 0), null:BIGINT, ($f3 / $f4)) AS $f3]) +- LocalGroupAggregate(groupBy=[a, $f2], partialFinalType=[PARTIAL], select=[a, $f2, COUNT(distinct$0 b) AS count$0, SUM(b) AS sum$1, $SUM0(b) AS sum$2, COUNT(b) AS count$3, DISTINCT(b) AS distinct$0]) +- Calc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -335,7 +335,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[COUNT()]) +- LogicalProject(a=[$1], b=[$2]) +- LogicalAggregate(group=[{0}], a=[AVG($1)], b=[AVG($2)]) +- LogicalProject(c=[$2], a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -350,7 +350,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RET +- Exchange(distribution=[hash[c]], changelogMode=[I]) +- LocalGroupAggregate(groupBy=[c], select=[c, AVG(a) AS (sum$0, count$1), AVG(b) AS (sum$2, count$3)], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -362,7 +362,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RET @@ -375,7 +375,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, FIRST_VALUE_RET +- Expand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -387,7 +387,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, FIRST_VALUE_RET @@ -400,7 +400,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, LAST_VALUE_RETR +- Expand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -412,7 +412,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, LAST_VALUE_RETR @@ -424,7 +424,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, LISTAGG(( +- LocalGroupAggregate(groupBy=[a, $f3], partialFinalType=[PARTIAL], select=[a, $f3, LISTAGG(c) AS (accDelimiter$0, concatAcc$1), COUNT(distinct$0 b) AS count$2, DISTINCT(b) AS distinct$0]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -439,7 +439,7 @@ GROUP BY a @@ -453,7 +453,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0(cou +- Expand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -476,7 +476,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT(DISTINCT $1)], EXP +- LogicalProject(b=[$1], b1=[$2]) +- LogicalAggregate(group=[{0}], b=[COUNT($1)], b1=[MAX($1)]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -492,7 +492,7 @@ GlobalGroupAggregate(groupBy=[b], partialFinalType=[FINAL], select=[b, SUM_RETRA +- LocalGroupAggregate(groupBy=[a], select=[a, COUNT(b) AS count$0, MAX(b) AS max$1], changelogMode=[I]) +- Calc(select=[a, b], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I]) ]]> @@ -504,7 +504,7 @@ GlobalGroupAggregate(groupBy=[b], partialFinalType=[FINAL], select=[b, SUM_RETRA @@ -516,7 +516,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0(cou +- LocalGroupAggregate(groupBy=[a], partialFinalType=[PARTIAL], select=[a, COUNT(distinct$0 a) AS count$0, COUNT(b) AS count$1, DISTINCT(a) AS distinct$0]) +- Calc(select=[a, b]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -528,7 +528,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0(cou @@ -542,7 +542,7 @@ GlobalGroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, SUM(sum$0 +- Expand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}, {a, b, c, null AS $f3, null AS $f4, 3 AS $e}]) +- Calc(select=[a, b, c, MOD(HASH_CODE(a), 1024) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.xml index eaab162d3f16e..090e814992e91 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.xml @@ -25,7 +25,7 @@ limitations under the License. LogicalProject(EXPR$0=[$1]) +- LogicalAggregate(group=[{0}], EXPR$0=[AVG($1)]) +- LogicalProject(b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -36,7 +36,7 @@ Calc(select=[EXPR$0]) +- LocalGroupAggregate(groupBy=[b], select=[b, AVG(a) AS (sum$0, count$1)]) +- Calc(select=[b, a]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -48,7 +48,7 @@ Calc(select=[EXPR$0]) @@ -58,7 +58,7 @@ GlobalGroupAggregate(select=[AVG((sum$0, count$1)) AS EXPR$0]), rowType=[RecordT +- LocalGroupAggregate(select=[AVG($f0) AS (sum$0, count$1)]), rowType=[RecordType(DOUBLE sum$0, BIGINT count$1)] +- Calc(select=[CAST(a AS DOUBLE) AS $f0]), rowType=[RecordType(DOUBLE $f0)] +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]), rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)] - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]), rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]), rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)] ]]> @@ -71,7 +71,7 @@ GlobalGroupAggregate(select=[AVG((sum$0, count$1)) AS EXPR$0]), rowType=[RecordT LogicalProject(EXPR$0=[$1]) +- LogicalAggregate(group=[{0}], EXPR$0=[COUNT($1)]) +- LogicalProject(b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -82,7 +82,7 @@ Calc(select=[EXPR$0]) +- LocalGroupAggregate(groupBy=[b], select=[b, COUNT(a) AS count$0]) +- Calc(select=[b, a]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -96,7 +96,7 @@ LogicalProject(EXPR$0=[$1], EXPR$1=[$2]) +- LogicalAggregate(group=[{0}], EXPR$0=[MIN($1)], EXPR$1=[AVG($2)]) +- LogicalProject(d=[$1], c=[$2], a=[$0]) +- LogicalProject(a=[$0], d=[+($1, 3)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -107,7 +107,7 @@ Calc(select=[EXPR$0, EXPR$1]) +- LocalGroupAggregate(groupBy=[d], select=[d, MIN(c) AS min$0, AVG(a) AS (sum$1, count$2)]) +- Calc(select=[(b + 3) AS d, c, a]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -119,7 +119,7 @@ Calc(select=[EXPR$0, EXPR$1]) @@ -129,7 +129,7 @@ GlobalGroupAggregate(select=[COUNT(count$0) AS EXPR$0]) +- LocalGroupAggregate(select=[COUNT(a) AS count$0]) +- Calc(select=[a]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -143,7 +143,7 @@ LogicalProject(b=[$0], EXPR$1=[$1]) +- LogicalFilter(condition=[=($0, 2)]) +- LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)]) +- LogicalProject(b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -154,7 +154,7 @@ Calc(select=[CAST(2 AS BIGINT) AS b, EXPR$1]) +- LocalGroupAggregate(groupBy=[b], select=[b, SUM(a) AS sum$0]) +- Calc(select=[CAST(2 AS BIGINT) AS b, a], where=[(b = 2)]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -167,7 +167,7 @@ Calc(select=[CAST(2 AS BIGINT) AS b, EXPR$1]) LogicalProject(four=[$1], EXPR$1=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[SUM($2)]) +- LogicalProject(b=[$1], four=[4], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -178,7 +178,7 @@ Calc(select=[4 AS four, EXPR$1]) +- LocalGroupAggregate(groupBy=[b], select=[b, SUM(a) AS sum$0]) +- Calc(select=[b, a]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.xml index a7774c145609a..ce391145efd97 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.xml @@ -24,8 +24,8 @@ limitations under the License. @@ -34,10 +34,10 @@ Calc(select=[a1, b1]) +- Join(joinType=[InnerJoin], where=[(((a1 = 1) AND (b1 = 1)) OR ((a2 = 2) AND (b2 = 2)))], select=[a1, a2, b1, b2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) : +- Calc(select=[a1, a2], where=[((1 = a1) OR (2 = a2))]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3]) +- Exchange(distribution=[single]) +- Calc(select=[b1, b2], where=[((1 = b1) OR (2 = b2))]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3]) ]]> @@ -49,8 +49,8 @@ Calc(select=[a1, b1]) @@ -58,10 +58,10 @@ LogicalProject(a=[$0], b=[$1], c=[$2], x=[$3], y=[$4], z=[$5]) Join(joinType=[InnerJoin], where=[(((a = 1) AND (x = 1)) OR ((a = 2) AND y IS NULL))], select=[a, b, c, x, y, z], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) : +- Calc(select=[a, b, c], where=[SEARCH(a, Sarg[1, 2])]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) +- Exchange(distribution=[single]) +- Calc(select=[x, y, z], where=[(y IS NULL OR (1 = x))]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, s, source: [TestTableSource(x, y, z)]]], fields=[x, y, z]) + +- TableSourceScan(table=[[default_catalog, default_database, s]], fields=[x, y, z]) ]]> @@ -73,8 +73,8 @@ Join(joinType=[InnerJoin], where=[(((a = 1) AND (x = 1)) OR ((a = 2) AND y IS NU @@ -83,10 +83,10 @@ Calc(select=[a1, b1]) +- Join(joinType=[InnerJoin], where=[(((a1 = 1) AND (b1 = 1)) OR (a2 = 2))], select=[a1, a2, b1], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) : +- Calc(select=[a1, a2], where=[((1 = a1) OR (2 = a2))]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3]) +- Exchange(distribution=[single]) +- Calc(select=[b1]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3]) ]]> @@ -98,8 +98,8 @@ Calc(select=[a1, b1]) @@ -107,10 +107,10 @@ LogicalProject(a1=[$0], b1=[$3]) Join(joinType=[FullOuterJoin], where=[=(a1, b1)], select=[a1, b1], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], changelogMode=[I,UA,D]) :- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1], changelogMode=[I]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) +: +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -122,8 +122,8 @@ Join(joinType=[FullOuterJoin], where=[=(a1, b1)], select=[a1, b1], leftInputSpec ($1, $4))], joinType=[full]) - :- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, A]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -132,10 +132,10 @@ Calc(select=[a1, b1], changelogMode=[I,UA,D]) +- Join(joinType=[FullOuterJoin], where=[AND(=(a1, b1), >(a2, b2))], select=[a1, a2, b1, b2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], changelogMode=[I,UA,D]) :- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -150,11 +150,11 @@ LogicalProject(a1=[$1], b1=[$3]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalProject(b2=[$1], b1=[$0]) +- LogicalAggregate(group=[{0}], b2=[SUM($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -164,12 +164,12 @@ Join(joinType=[FullOuterJoin], where=[=(a1, b1)], select=[a1, b1], leftInputSpec : +- GroupAggregate(groupBy=[a1], select=[a1], changelogMode=[I,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) +: +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b1]], changelogMode=[I,UA]) +- GroupAggregate(groupBy=[b1], select=[b1], changelogMode=[I,UA]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -184,11 +184,11 @@ LogicalProject(a1=[$1], b1=[$3]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalProject(b2=[$1], b1=[$0]) +- LogicalAggregate(group=[{0}], b2=[SUM($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -200,13 +200,13 @@ Calc(select=[a1, b1], changelogMode=[I,UA,D]) : +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], changelogMode=[I,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b1]], changelogMode=[I,UA]) +- Calc(select=[b2, b1], changelogMode=[I,UA]) +- GroupAggregate(groupBy=[b1], select=[b1, SUM(b2) AS b2], changelogMode=[I,UA]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -221,8 +221,8 @@ LogicalProject(a1=[$1], b1=[$2]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -232,10 +232,10 @@ Join(joinType=[FullOuterJoin], where=[=(a1, b1)], select=[a1, b1], leftInputSpec : +- GroupAggregate(groupBy=[a1], select=[a1], changelogMode=[I,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) +: +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -250,8 +250,8 @@ LogicalProject(a1=[$1], b1=[$2]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -263,10 +263,10 @@ Calc(select=[a1, b1], changelogMode=[I,UA,D]) : +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], changelogMode=[I,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -281,11 +281,11 @@ LogicalProject(a1=[$1], a2=[$0], b1=[$3], b2=[$2]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalProject(b2=[$1], b1=[$0]) +- LogicalAggregate(group=[{0}], b2=[SUM($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -297,13 +297,13 @@ Calc(select=[a1, a2, b1, b2], changelogMode=[I,UB,UA,D]) : +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], changelogMode=[I,UB,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b2]], changelogMode=[I,UB,UA]) +- Calc(select=[b2, b1], changelogMode=[I,UB,UA]) +- GroupAggregate(groupBy=[b1], select=[b1, SUM(b2) AS b2], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -318,11 +318,11 @@ LogicalProject(a1=[$1], a2=[$0], b1=[$3], b2=[$2]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalProject(b2=[$1], b1=[$0]) +- LogicalAggregate(group=[{0}], b2=[SUM($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -334,13 +334,13 @@ Calc(select=[a1, a2, b1, b2], changelogMode=[I,UB,UA,D]) : +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], changelogMode=[I,UB,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b2]], changelogMode=[I,UB,UA]) +- Calc(select=[b2, b1], changelogMode=[I,UB,UA]) +- GroupAggregate(groupBy=[b1], select=[b1, SUM(b2) AS b2], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -352,8 +352,8 @@ Calc(select=[a1, a2, b1, b2], changelogMode=[I,UB,UA,D]) @@ -361,10 +361,10 @@ LogicalProject(a1=[$0], b1=[$3]) Join(joinType=[InnerJoin], where=[(a1 = b1)], select=[a1, b1], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a1]]) : +- Calc(select=[a1]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3]) +: +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3]) +- Exchange(distribution=[hash[b1]]) +- Calc(select=[b1]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3]) ]]> @@ -379,11 +379,11 @@ LogicalProject(a1=[$1], b1=[$3]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalProject(b2=[$1], b1=[$0]) +- LogicalAggregate(group=[{0}], b2=[SUM($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -393,12 +393,12 @@ Join(joinType=[InnerJoin], where=[=(a1, b1)], select=[a1, b1], leftInputSpec=[Jo : +- GroupAggregate(groupBy=[a1], select=[a1], changelogMode=[I,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) +: +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b1]], changelogMode=[I,UA]) +- GroupAggregate(groupBy=[b1], select=[b1], changelogMode=[I,UA]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -419,10 +419,10 @@ LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalAggregate(group=[{0}], a2=[COUNT($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalAggregate(group=[{0}], b2=[COUNT($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -433,12 +433,12 @@ Calc(select=[a1, a2, CAST(2 AS INTEGER) AS b1, b2]) : +- GroupAggregate(groupBy=[a1], select=[a1, COUNT(a2) AS a2]) : +- Exchange(distribution=[hash[a1]]) : +- Calc(select=[CAST(2 AS INTEGER) AS a1, a2], where=[(a1 = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3]) +- Exchange(distribution=[hash[b1, b2]]) +- GroupAggregate(groupBy=[b1], select=[b1, COUNT(b2) AS b2]) +- Exchange(distribution=[hash[b1]]) +- Calc(select=[CAST(2 AS INTEGER) AS b1, b2], where=[(b1 = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3]) ]]> @@ -459,10 +459,10 @@ LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalAggregate(group=[{0}], a2=[COUNT($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalAggregate(group=[{0}], b2=[COUNT($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -474,13 +474,13 @@ Calc(select=[a1, 1 AS a2, CAST(2 AS INTEGER) AS b1, 1 AS b2]) : +- GroupAggregate(groupBy=[a1], select=[a1, COUNT(a2) AS a2]) : +- Exchange(distribution=[hash[a1]]) : +- Calc(select=[CAST(2 AS INTEGER) AS a1, a2], where=[(a1 = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3]) +- Exchange(distribution=[hash[b1]]) +- Calc(select=[b1], where=[(b2 = 1)]) +- GroupAggregate(groupBy=[b1], select=[b1, COUNT(b2) AS b2]) +- Exchange(distribution=[hash[b1]]) +- Calc(select=[CAST(2 AS INTEGER) AS b1, b2], where=[(b1 = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3]) ]]> @@ -495,11 +495,11 @@ LogicalProject(a1=[$1], b1=[$3]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalProject(b2=[$1], b1=[$0]) +- LogicalAggregate(group=[{0}], b2=[SUM($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -509,12 +509,12 @@ Join(joinType=[LeftOuterJoin], where=[=(a1, b1)], select=[a1, b1], leftInputSpec : +- GroupAggregate(groupBy=[a1], select=[a1], changelogMode=[I,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) +: +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b1]], changelogMode=[I,UA]) +- GroupAggregate(groupBy=[b1], select=[b1], changelogMode=[I,UA]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -529,11 +529,11 @@ LogicalProject(a1=[$1], a2=[$0], b1=[$3], b2=[$2]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalProject(b2=[$1], b1=[$0]) +- LogicalAggregate(group=[{0}], b2=[SUM($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -545,13 +545,13 @@ Calc(select=[a1, a2, b1, b2], changelogMode=[I,UB,UA]) : +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], changelogMode=[I,UB,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b2]], changelogMode=[I,UB,UA]) +- Calc(select=[b2, b1], changelogMode=[I,UB,UA]) +- GroupAggregate(groupBy=[b1], select=[b1, SUM(b2) AS b2], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -563,7 +563,7 @@ Calc(select=[a1, a2, b1, b2], changelogMode=[I,UB,UA]) @@ -573,7 +573,7 @@ Calc(select=[a1]) +- Join(joinType=[LeftOuterJoin], where=[(a1 = pk1)], select=[a1, pk1], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a1]]) : +- Calc(select=[a1]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3]) +- Exchange(distribution=[hash[pk1]]) +- TableSourceScan(table=[[default_catalog, default_database, tableWithCompositePk, project=[pk1], metadata=[]]], fields=[pk1]) ]]> @@ -824,18 +824,18 @@ SELECT * FROM @@ -847,8 +847,8 @@ Join(joinType=[FullOuterJoin], where=[((i = i0) AND (i = k))], select=[i, j, t, @@ -856,10 +856,10 @@ LogicalProject(a1=[$0], b1=[$3]) Join(joinType=[LeftOuterJoin], where=[=(a1, b1)], select=[a1, b1], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], changelogMode=[I,UA,D]) :- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1], changelogMode=[I]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) +: +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -871,8 +871,8 @@ Join(joinType=[LeftOuterJoin], where=[=(a1, b1)], select=[a1, b1], leftInputSpec ($1, $4))], joinType=[left]) - :- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, A]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -881,10 +881,10 @@ Calc(select=[a1, b1], changelogMode=[I,UA,D]) +- Join(joinType=[LeftOuterJoin], where=[AND(=(a1, b1), >(a2, b2))], select=[a1, a2, b1, b2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], changelogMode=[I,UA,D]) :- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -899,11 +899,11 @@ LogicalProject(a1=[$1], b1=[$3]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalProject(b2=[$1], b1=[$0]) +- LogicalAggregate(group=[{0}], b2=[SUM($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -915,13 +915,13 @@ Calc(select=[a1, b1], changelogMode=[I,UA,D]) : +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], changelogMode=[I,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b1]], changelogMode=[I,UA]) +- Calc(select=[b2, b1], changelogMode=[I,UA]) +- GroupAggregate(groupBy=[b1], select=[b1, SUM(b2) AS b2], changelogMode=[I,UA]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -942,10 +942,10 @@ LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3]) +- LogicalJoin(condition=[true], joinType=[left]) :- LogicalAggregate(group=[{0}], a2=[COUNT($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalAggregate(group=[{0}], b2=[COUNT($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -956,12 +956,12 @@ Calc(select=[CAST(2 AS INTEGER) AS a1, a2, b1, CAST(b2 AS BIGINT) AS b2]) : +- GroupAggregate(groupBy=[a1], select=[a1, COUNT(a2) AS a2]) : +- Exchange(distribution=[hash[a1]]) : +- Calc(select=[CAST(2 AS INTEGER) AS a1, a2], where=[(a1 = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3]) +- Exchange(distribution=[hash[b1, b2]]) +- GroupAggregate(groupBy=[b1], select=[b1, COUNT(b2) AS b2]) +- Exchange(distribution=[hash[b1]]) +- Calc(select=[CAST(2 AS INTEGER) AS b1, b2], where=[(b1 = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3]) ]]> @@ -981,10 +981,10 @@ LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3]) +- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($0, 2), =($3, 1))], joinType=[left]) :- LogicalAggregate(group=[{0}], a2=[COUNT($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalAggregate(group=[{0}], b2=[COUNT($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -994,13 +994,13 @@ Join(joinType=[LeftOuterJoin], where=[((a1 = b1) AND (a2 = 1) AND (a1 = 2))], se : +- GroupAggregate(groupBy=[a1], select=[a1, COUNT(a2) AS a2]) : +- Exchange(distribution=[hash[a1]]) : +- Calc(select=[a1, a2]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3]) +: +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3]) +- Exchange(distribution=[hash[b1]]) +- Calc(select=[b1, b2], where=[(b2 = 1)]) +- GroupAggregate(groupBy=[b1], select=[b1, COUNT(b2) AS b2]) +- Exchange(distribution=[hash[b1]]) +- Calc(select=[b1, b2]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3]) ]]> @@ -1015,11 +1015,11 @@ LogicalProject(a1=[$1], a2=[$0], b1=[$3], b2=[$2]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalProject(b2=[$1], b1=[$0]) +- LogicalAggregate(group=[{0}], b2=[SUM($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -1031,13 +1031,13 @@ Calc(select=[a1, a2, b1, b2], changelogMode=[I,UB,UA,D]) : +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], changelogMode=[I,UB,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b2]], changelogMode=[I,UB,UA]) +- Calc(select=[b2, b1], changelogMode=[I,UB,UA]) +- GroupAggregate(groupBy=[b1], select=[b1, SUM(b2) AS b2], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -1052,11 +1052,11 @@ LogicalProject(a1=[$1], a2=[$0], b1=[$3], b2=[$2]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalProject(b2=[$1], b1=[$0]) +- LogicalAggregate(group=[{0}], b2=[SUM($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -1068,13 +1068,13 @@ Calc(select=[a1, a2, b1, b2], changelogMode=[I,UB,UA,D]) : +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], changelogMode=[I,UB,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b2]], changelogMode=[I,UB,UA]) +- Calc(select=[b2, b1], changelogMode=[I,UB,UA]) +- GroupAggregate(groupBy=[b1], select=[b1, SUM(b2) AS b2], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -1089,8 +1089,8 @@ LogicalProject(a1=[$1], b1=[$2]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -1100,10 +1100,10 @@ Join(joinType=[LeftOuterJoin], where=[=(a1, b1)], select=[a1, b1], leftInputSpec : +- GroupAggregate(groupBy=[a1], select=[a1], changelogMode=[I,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) +: +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -1118,8 +1118,8 @@ LogicalProject(a1=[$1], b1=[$2]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -1131,10 +1131,10 @@ Calc(select=[a1, b1], changelogMode=[I,UA,D]) : +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], changelogMode=[I,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -1146,8 +1146,8 @@ Calc(select=[a1, b1], changelogMode=[I,UA,D]) @@ -1156,10 +1156,10 @@ Calc(select=[b, y]) +- Join(joinType=[LeftOuterJoin], where=[((a = z) AND (b < 2))], select=[a, b, y, z], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) +- Exchange(distribution=[hash[z]]) +- Calc(select=[y, z]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, s, source: [TestTableSource(x, y, z)]]], fields=[x, y, z]) + +- TableSourceScan(table=[[default_catalog, default_database, s]], fields=[x, y, z]) ]]> @@ -1171,8 +1171,8 @@ Calc(select=[b, y]) @@ -1181,9 +1181,9 @@ Calc(select=[b, y]) +- Join(joinType=[LeftOuterJoin], where=[((a = z) AND (b < x))], select=[a, b, x, y, z], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) +- Exchange(distribution=[hash[z]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, s, source: [TestTableSource(x, y, z)]]], fields=[x, y, z]) + +- TableSourceScan(table=[[default_catalog, default_database, s]], fields=[x, y, z]) ]]> @@ -1195,8 +1195,8 @@ Calc(select=[b, y]) ($1, $4))], joinType=[right]) - :- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, A]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -1205,10 +1205,10 @@ Calc(select=[a1, b1], changelogMode=[I,UA,D]) +- Join(joinType=[RightOuterJoin], where=[AND(=(a1, b1), >(a2, b2))], select=[a1, a2, b1, b2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], changelogMode=[I,UA,D]) :- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -1220,8 +1220,8 @@ Calc(select=[a1, b1], changelogMode=[I,UA,D]) @@ -1230,10 +1230,10 @@ Calc(select=[b, y]) +- Join(joinType=[LeftOuterJoin], where=[(a = z)], select=[a, b, y, z], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) +- Exchange(distribution=[hash[z]]) +- Calc(select=[y, z]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, s, source: [TestTableSource(x, y, z)]]], fields=[x, y, z]) + +- TableSourceScan(table=[[default_catalog, default_database, s]], fields=[x, y, z]) ]]> @@ -1245,8 +1245,8 @@ Calc(select=[b, y]) @@ -1254,10 +1254,10 @@ LogicalProject(a1=[$0], b1=[$3]) Join(joinType=[RightOuterJoin], where=[=(a1, b1)], select=[a1, b1], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], changelogMode=[I,UA,D]) :- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1], changelogMode=[I]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) +: +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -1272,11 +1272,11 @@ LogicalProject(a1=[$1], b1=[$3]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalProject(b2=[$1], b1=[$0]) +- LogicalAggregate(group=[{0}], b2=[SUM($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -1286,12 +1286,12 @@ Join(joinType=[RightOuterJoin], where=[=(a1, b1)], select=[a1, b1], leftInputSpe : +- GroupAggregate(groupBy=[a1], select=[a1], changelogMode=[I,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) +: +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b1]], changelogMode=[I,UA]) +- GroupAggregate(groupBy=[b1], select=[b1], changelogMode=[I,UA]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -1306,11 +1306,11 @@ LogicalProject(a1=[$1], b1=[$3]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalProject(b2=[$1], b1=[$0]) +- LogicalAggregate(group=[{0}], b2=[SUM($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -1322,13 +1322,13 @@ Calc(select=[a1, b1], changelogMode=[I,UA,D]) : +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], changelogMode=[I,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b1]], changelogMode=[I,UA]) +- Calc(select=[b2, b1], changelogMode=[I,UA]) +- GroupAggregate(groupBy=[b1], select=[b1, SUM(b2) AS b2], changelogMode=[I,UA]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -1349,10 +1349,10 @@ LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3]) +- LogicalJoin(condition=[true], joinType=[right]) :- LogicalAggregate(group=[{0}], a2=[COUNT($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalAggregate(group=[{0}], b2=[COUNT($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -1363,12 +1363,12 @@ Calc(select=[a1, CAST(a2 AS BIGINT) AS a2, CAST(2 AS INTEGER) AS b1, b2]) : +- GroupAggregate(groupBy=[a1], select=[a1, COUNT(a2) AS a2]) : +- Exchange(distribution=[hash[a1]]) : +- Calc(select=[CAST(2 AS INTEGER) AS a1, a2], where=[(a1 = 2)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3]) +- Exchange(distribution=[hash[b1, b2]]) +- GroupAggregate(groupBy=[b1], select=[b1, COUNT(b2) AS b2]) +- Exchange(distribution=[hash[b1]]) +- Calc(select=[CAST(2 AS INTEGER) AS b1, b2], where=[(b1 = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3]) ]]> @@ -1380,8 +1380,8 @@ Calc(select=[a1, CAST(a2 AS BIGINT) AS a2, CAST(2 AS INTEGER) AS b1, b2]) @@ -1390,10 +1390,10 @@ Calc(select=[b, y]) +- Join(joinType=[RightOuterJoin], where=[(a = z)], select=[a, b, y, z], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) +- Exchange(distribution=[hash[z]]) +- Calc(select=[y, z]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, s, source: [TestTableSource(x, y, z)]]], fields=[x, y, z]) + +- TableSourceScan(table=[[default_catalog, default_database, s]], fields=[x, y, z]) ]]> @@ -1413,10 +1413,10 @@ LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3]) +- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($2, 2), =($1, 1))], joinType=[right]) :- LogicalAggregate(group=[{0}], a2=[COUNT($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalAggregate(group=[{0}], b2=[COUNT($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -1427,12 +1427,12 @@ Join(joinType=[RightOuterJoin], where=[((a1 = b1) AND (1 = b2) AND (b1 = 2))], s : +- GroupAggregate(groupBy=[a1], select=[a1, COUNT(a2) AS a2]) : +- Exchange(distribution=[hash[a1]]) : +- Calc(select=[a1, a2]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3]) +: +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3]) +- Exchange(distribution=[hash[b1]]) +- GroupAggregate(groupBy=[b1], select=[b1, COUNT(b2) AS b2]) +- Exchange(distribution=[hash[b1]]) +- Calc(select=[b1, b2]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3]) ]]> @@ -1447,11 +1447,11 @@ LogicalProject(a1=[$1], a2=[$0], b1=[$3], b2=[$2]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalProject(b2=[$1], b1=[$0]) +- LogicalAggregate(group=[{0}], b2=[SUM($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -1463,13 +1463,13 @@ Calc(select=[a1, a2, b1, b2], changelogMode=[I,UB,UA,D]) : +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], changelogMode=[I,UB,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b2]], changelogMode=[I,UB,UA]) +- Calc(select=[b2, b1], changelogMode=[I,UB,UA]) +- GroupAggregate(groupBy=[b1], select=[b1, SUM(b2) AS b2], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -1484,11 +1484,11 @@ LogicalProject(a1=[$1], a2=[$0], b1=[$3], b2=[$2]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalProject(b2=[$1], b1=[$0]) +- LogicalAggregate(group=[{0}], b2=[SUM($1)]) +- LogicalProject(b1=[$0], b2=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -1500,13 +1500,13 @@ Calc(select=[a1, a2, b1, b2], changelogMode=[I,UB,UA,D]) : +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], changelogMode=[I,UB,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b2]], changelogMode=[I,UB,UA]) +- Calc(select=[b2, b1], changelogMode=[I,UB,UA]) +- GroupAggregate(groupBy=[b1], select=[b1, SUM(b2) AS b2], changelogMode=[I,UB,UA]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -1521,8 +1521,8 @@ LogicalProject(a1=[$1], b1=[$2]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -1532,10 +1532,10 @@ Join(joinType=[RightOuterJoin], where=[=(a1, b1)], select=[a1, b1], leftInputSpe : +- GroupAggregate(groupBy=[a1], select=[a1], changelogMode=[I,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) +: +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -1550,8 +1550,8 @@ LogicalProject(a1=[$1], b1=[$2]) :- LogicalProject(a2=[$1], a1=[$0]) : +- LogicalAggregate(group=[{0}], a2=[SUM($1)]) : +- LogicalProject(a1=[$0], a2=[$1]) - : +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, A]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -1563,10 +1563,10 @@ Calc(select=[a1, b1], changelogMode=[I,UA,D]) : +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], changelogMode=[I,UA]) : +- Exchange(distribution=[hash[a1]], changelogMode=[I]) : +- Calc(select=[a1, a2], changelogMode=[I]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I]) + : +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2, a3], changelogMode=[I]) +- Exchange(distribution=[hash[b1]], changelogMode=[I]) +- Calc(select=[b1, b2], changelogMode=[I]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2, b3], changelogMode=[I]) ]]> @@ -1578,8 +1578,8 @@ Calc(select=[a1, b1], changelogMode=[I,UA,D]) @@ -1588,10 +1588,10 @@ Calc(select=[b, x]) +- Join(joinType=[RightOuterJoin], where=[((a = z) AND (x < 2))], select=[a, b, x, z], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) +- Exchange(distribution=[hash[z]]) +- Calc(select=[x, z]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, s, source: [TestTableSource(x, y, z)]]], fields=[x, y, z]) + +- TableSourceScan(table=[[default_catalog, default_database, s]], fields=[x, y, z]) ]]> @@ -1603,8 +1603,8 @@ Calc(select=[b, x]) @@ -1613,9 +1613,9 @@ Calc(select=[b, y]) +- Join(joinType=[RightOuterJoin], where=[((a = z) AND (b < x))], select=[a, b, x, y, z], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b, c]) +- Exchange(distribution=[hash[z]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, s, source: [TestTableSource(x, y, z)]]], fields=[x, y, z]) + +- TableSourceScan(table=[[default_catalog, default_database, s]], fields=[x, y, z]) ]]> @@ -1635,16 +1635,16 @@ LogicalProject(key=[$0], v=[$1], key0=[$2], v0=[$3]) +- LogicalJoin(condition=[AND(=($0, $2), >($2, 10))], joinType=[left]) :- LogicalProject(key=[$0], v=[$1]) : +- LogicalFilter(condition=[=($0, 0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(key, v)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, src]]) +- LogicalProject(key=[$0], v=[$1]) +- LogicalFilter(condition=[=($0, 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(key, v)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, src]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.xml index 37119efe848f9..93f8ca912f590 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.xml @@ -24,12 +24,12 @@ limitations under the License. @@ -38,16 +38,16 @@ Join(joinType=[LeftAntiJoin], where=[(a = i)], select=[a, b, c], leftInputSpec=[ :- Exchange(distribution=[hash[a]]) : +- Join(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) : :- Exchange(distribution=[single]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[single]) : +- Calc(select=[m IS NOT NULL AS $f0]) : +- GroupAggregate(select=[MIN(i) AS m]) : +- Exchange(distribution=[single]) : +- Calc(select=[true AS i]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[i]]) +- Calc(select=[i], where=[(j < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -62,22 +62,22 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$0=[MAX($4)]) LogicalProject(d=[$0], $f1=[true], f=[$2], $f3=[1], e=[$1]) LogicalFilter(condition=[AND(=($cor0.b, $1), <($0, 100), =($cor0.c, $2))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -91,21 +91,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=($cor1.a, $0)]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })], variablesSet=[[$cor1]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -118,19 +118,19 @@ Join(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], leftInputSpec=[ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[<($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -143,9 +143,9 @@ Join(joinType=[LeftSemiJoin], where=[(a < d)], select=[a, b, c], leftInputSpec=[ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[<($cor0.a, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -153,10 +153,10 @@ LogicalFilter(condition=[<($cor0.a, 10)]) Join(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) : +- Calc(select=[a, b, c], where=[(a < 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[single]) +- Calc(select=[]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -171,22 +171,22 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalAggregate(group=[{0}]) LogicalProject(e=[$1]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -199,19 +199,19 @@ Join(joinType=[LeftSemiJoin], where=[(c = f)], select=[a, b, c], leftInputSpec=[ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -223,9 +223,9 @@ Join(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], leftInputSpec=[ ($1, 10), NOT(OR(LIKE($2, _UTF-16LE'abc'), NOT(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) +LogicalTableScan(table=[[default_catalog, default_database, r]]) })))))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -234,13 +234,13 @@ Calc(select=[(a + 10) AS EXPR$0, c]) +- Join(joinType=[LeftSemiJoin], where=[true], select=[a, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) : +- Calc(select=[a, c], where=[((b > 10) AND NOT(LIKE(c, 'abc')))]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[single]) +- Calc(select=[], where=[m IS NOT NULL]) +- GroupAggregate(select=[MIN(i) AS m]) +- Exchange(distribution=[single]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -253,12 +253,12 @@ Calc(select=[(a + 10) AS EXPR$0, c]) LogicalProject(c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalJoin(condition=[=($1, $3)], joinType=[full]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -266,7 +266,7 @@ LogicalJoin(condition=[=($1, $3)], joinType=[full]) Join(joinType=[LeftSemiJoin], where=[true], select=[c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) : +- Calc(select=[c]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[single]) +- Calc(select=[], where=[m IS NOT NULL]) +- GroupAggregate(select=[MIN(i) AS m]) @@ -275,10 +275,10 @@ Join(joinType=[LeftSemiJoin], where=[true], select=[c], leftInputSpec=[NoUniqueK +- Join(joinType=[FullOuterJoin], where=[(e = j)], select=[e, j], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[e]]) : +- Calc(select=[e]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j]]) +- Calc(select=[j], where=[(i > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -291,20 +291,20 @@ Join(joinType=[LeftSemiJoin], where=[true], select=[c], leftInputSpec=[NoUniqueK LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) +LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -332,22 +332,22 @@ Join(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], leftInputSpec=[NoU @@ -359,9 +359,9 @@ Join(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], leftInputSpec=[NoU ($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -369,13 +369,13 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl Join(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) : +- Calc(select=[a, b, c], where=[(b > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[single]) +- Calc(select=[], where=[m IS NOT NULL]) +- GroupAggregate(select=[MIN(i) AS m]) +- Exchange(distribution=[single]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -390,12 +390,12 @@ LogicalProject(a=[$0]) LogicalUnion(all=[false]) LogicalProject(e=[$1], f=[$2]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1], k=[$2]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -403,7 +403,7 @@ LogicalUnion(all=[false]) Join(joinType=[LeftSemiJoin], where=[true], select=[a], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) : +- Calc(select=[a]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[single]) +- Calc(select=[], where=[m IS NOT NULL]) +- GroupAggregate(select=[MIN(i) AS m]) @@ -413,9 +413,9 @@ Join(joinType=[LeftSemiJoin], where=[true], select=[a], leftInputSpec=[NoUniqueK +- Exchange(distribution=[hash[e, f]]) +- Union(all=[true], union=[e, f]) :- Calc(select=[e, f], where=[(d > 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Calc(select=[j, k], where=[(i < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -428,13 +428,13 @@ Join(joinType=[LeftSemiJoin], where=[true], select=[a], leftInputSpec=[NoUniqueK LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -443,13 +443,13 @@ Join(joinType=[LeftSemiJoin], where=[((a = i) AND (b = j))], select=[a, b, c], l :- Exchange(distribution=[hash[a, b]]) : +- Join(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) : :- Exchange(distribution=[hash[a]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[i, j]]) +- Calc(select=[i, j]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -462,12 +462,12 @@ Join(joinType=[LeftSemiJoin], where=[((a = i) AND (b = j))], select=[a, b, c], l LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -476,13 +476,13 @@ Join(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], leftInputSpec=[ :- Exchange(distribution=[hash[a]]) : +- Join(joinType=[LeftSemiJoin], where=[(b = j)], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) : :- Exchange(distribution=[hash[b]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[j]]) : +- Calc(select=[j]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -495,21 +495,21 @@ Join(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], leftInputSpec=[ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), NOT(IN($1, { LogicalProject(i=[$0]) LogicalFilter(condition=[AND(=($cor0.c, $2), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), EXISTS({ LogicalAggregate(group=[{0}], EXPR$0=[COUNT($0)]) LogicalProject(l=[$0]) LogicalFilter(condition=[LIKE($2, _UTF-16LE'Test')]) - LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t2]]) }), NOT(EXISTS({ LogicalFilter(condition=[<>($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -522,10 +522,10 @@ Join(joinType=[LeftAntiJoin], where=[(b <> e)], select=[a, b, c], leftInputSpec= : : :- Exchange(distribution=[single]) : : : +- Join(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) : : : :- Exchange(distribution=[hash[a]]) -: : : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : : : +- Exchange(distribution=[hash[d]]) : : : +- Calc(select=[d]) -: : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])(reuse_id=[1]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f])(reuse_id=[1]) : : +- Exchange(distribution=[single]) : : +- Calc(select=[true AS $f0], where=[m IS NOT NULL]) : : +- GroupAggregate(select=[MIN(i) AS m]) @@ -534,10 +534,10 @@ Join(joinType=[LeftAntiJoin], where=[(b <> e)], select=[a, b, c], leftInputSpec= : : +- GroupAggregate(groupBy=[l], select=[l]) : : +- Exchange(distribution=[hash[l]]) : : +- Calc(select=[l], where=[LIKE(n, 'Test')]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]], fields=[l, m, n]) +: : +- TableSourceScan(table=[[default_catalog, default_database, t2]], fields=[l, m, n]) : +- Exchange(distribution=[hash[k]]) : +- Calc(select=[i, k], where=[(i > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) +- Exchange(distribution=[single]) +- Calc(select=[e]) +- Reused(reference_id=[1]) @@ -556,22 +556,22 @@ LogicalProject(EXPR$0=[$1]) LogicalAggregate(group=[{0}], EXPR$0=[MAX($1)]) LogicalProject(f=[$2], e=[$1]) LogicalFilter(condition=[AND(=($cor0.c, $2), <($0, 3))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -587,21 +587,21 @@ LogicalProject(EXPR$0=[$4], d=[$0]) LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$0=[COUNT()]) LogicalProject(d=[$0], $f1=[true], e=[$1], $f3=[1]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -615,9 +615,9 @@ LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) +- LogicalFilter(condition=[IN(SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=(+($cor0.b, 1), $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -626,10 +626,10 @@ Calc(select=[(a + 10) AS EXPR$0, c]) +- Join(joinType=[LeftSemiJoin], where=[(($f3 = EXPR$0) AND ($f4 = e))], select=[a, c, $f3, $f4], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[$f3, $f4]]) : +- Calc(select=[a, c, SUBSTRING(c, 1, 5) AS $f3, (b + 1) AS $f4]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[EXPR$0, e]]) +- Calc(select=[SUBSTRING(f, 1, 5) AS EXPR$0, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -643,9 +643,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), NOT(LIKE($2, _UTF-16LE'abc')), IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -653,10 +653,10 @@ LogicalProject(d=[$0]) Join(joinType=[LeftSemiJoin], where=[((a = d) AND (b = e))], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a, b]]) : +- Calc(select=[a, b, c], where=[((b > 10) AND NOT(LIKE(c, 'abc')))]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, e]]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -669,13 +669,13 @@ Join(joinType=[LeftSemiJoin], where=[((a = d) AND (b = e))], select=[a, b, c], l LogicalProject(b=[$1]) +- LogicalFilter(condition=[IN(CASE(IN($0, { LogicalProject(EXPR$0=[1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), 1, 2), { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -688,11 +688,11 @@ Calc(select=[b]) : :- Exchange(distribution=[single]) : : +- Join(joinType=[InnerJoin], where=[true], select=[a, b, c, c0], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) : : :- Exchange(distribution=[single]) - : : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : : +- Exchange(distribution=[single]) : : +- GroupAggregate(select=[COUNT(*) AS c]) : : +- Exchange(distribution=[single]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) + : : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) : +- Exchange(distribution=[single]) : +- Calc(select=[true AS i]) : +- GroupAggregate(groupBy=[EXPR$0], select=[EXPR$0]) @@ -701,7 +701,7 @@ Calc(select=[b]) : +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[d, f]]) +- Calc(select=[d, f]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -717,12 +717,12 @@ LogicalProject(e=[$0]) LogicalJoin(condition=[=($0, $1)], joinType=[inner]) LogicalProject(e=[$1]) LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(j=[$1]) LogicalFilter(condition=[AND(=($cor0.c, $2), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -730,16 +730,16 @@ LogicalProject(e=[$0]) Calc(select=[c]) +- Join(joinType=[LeftSemiJoin], where=[((b = e) AND (a = d) AND (c = k))], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[b, a, c]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e, d, k]]) +- Calc(select=[e, d, k]) +- Join(joinType=[InnerJoin], where=[(e = j)], select=[e, d, j, k], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[e]]) : +- Calc(select=[e, d], where=[(e < 50)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j]]) +- Calc(select=[j, k], where=[(i > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -755,10 +755,10 @@ LogicalProject(j=[$3]) LogicalJoin(condition=[=($1, $4)], joinType=[left]) LogicalProject(e=[$1], f=[$2]) LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -766,16 +766,16 @@ LogicalProject(j=[$3]) Calc(select=[c]) +- Join(joinType=[LeftSemiJoin], where=[((b = j) AND (a = d))], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[b, a]]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[j, d]]) +- Calc(select=[j, d]) +- Join(joinType=[LeftOuterJoin], where=[(f = k)], select=[f, d, j, k], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[f]]) : +- Calc(select=[f, d], where=[(e < 50)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[k]]) +- Calc(select=[j, k]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -789,12 +789,12 @@ LogicalProject(c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalJoin(condition=[=($0, $3)], joinType=[right]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0]) LogicalFilter(condition=[AND(=($cor0.c, $2), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -803,16 +803,16 @@ Calc(select=[c]) +- Join(joinType=[LeftSemiJoin], where=[((a = d) AND (c = k))], select=[a, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a, c]]) : +- Calc(select=[a, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, k]]) +- Calc(select=[d, k]) +- Join(joinType=[RightOuterJoin], where=[(d = i)], select=[d, i, k], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[i]]) +- Calc(select=[i, k], where=[(i > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -827,21 +827,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(f1=[$3]) LogicalFilter(condition=[=($cor1.a, $0)]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })], variablesSet=[[$cor1]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -855,9 +855,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, SUBSTRING($2, 1, 5), { LogicalProject(d=[$0], EXPR$1=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -866,10 +866,10 @@ Calc(select=[a, b, c]) +- Join(joinType=[LeftSemiJoin], where=[((a = d) AND ($f3 = EXPR$1) AND (b = e))], select=[a, b, c, $f3], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a, $f3, b]]) : +- Calc(select=[a, b, c, SUBSTRING(c, 1, 5) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, EXPR$1, e]]) +- Calc(select=[d, SUBSTRING(f, 1, 5) AS EXPR$1, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -883,19 +883,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> e))], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -909,9 +909,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($cor0.b, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -919,10 +919,10 @@ LogicalProject(d=[$0]) Join(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[(b > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -937,21 +937,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(rk=[$2]) LogicalFilter(condition=[<>($cor0.a, $0)]) LogicalProject(d=[$0], e=[$1], rk=[RANK() OVER (PARTITION BY $0 ORDER BY $1 NULLS FIRST)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> d))], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[b]]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[rk]]) +- Calc(select=[w0$o0 AS rk, d]) +- OverAggregate(partitionBy=[d], orderBy=[e ASC], window=[ RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[d, e, f, RANK(*) AS w0$o0]) +- Exchange(distribution=[hash[d]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -967,15 +967,15 @@ LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) LogicalProject(d=[$0]) LogicalFilter(condition=[IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), { LogicalProject(i=[$0]) LogicalFilter(condition=[=($2, $cor0.c)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -986,7 +986,7 @@ Calc(select=[a]) : +- Join(joinType=[LeftOuterJoin], where=[true], select=[a, c, EXPR$0], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) : :- Exchange(distribution=[single]) : : +- Calc(select=[a, c]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[single]) : +- GroupAggregate(select=[MAX(d) AS EXPR$0]) : +- Exchange(distribution=[single]) @@ -994,12 +994,12 @@ Calc(select=[a]) : +- Join(joinType=[InnerJoin], where=[(e = j)], select=[d, e, j], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) : :- Exchange(distribution=[hash[e]]) : : +- Calc(select=[d, e]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) : +- Exchange(distribution=[hash[j]]) : +- GroupAggregate(groupBy=[j], select=[j]) : +- Exchange(distribution=[hash[j]]) : +- Calc(select=[j]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) +- Exchange(distribution=[hash[i, k]]) +- Calc(select=[i, k]) +- Reused(reference_id=[1]) @@ -1016,19 +1016,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1042,9 +1042,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 1), IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1052,10 +1052,10 @@ LogicalProject(d=[$0]) Join(joinType=[LeftSemiJoin], where=[((a = d) AND (b = e))], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a, b]]) : +- Calc(select=[a, b, c], where=[(b > 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, e]]) +- Calc(select=[d, e], where=[(d > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1069,9 +1069,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=(CAST($cor0.b):INTEGER, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1080,10 +1080,10 @@ Calc(select=[a, b, c]) +- Join(joinType=[LeftSemiJoin], where=[((a = d) AND (b0 = d))], select=[a, b, c, b0], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a, b0]]) : +- Calc(select=[a, b, c, CAST(b AS INTEGER) AS b0]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d, d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1099,9 +1099,9 @@ LogicalProject(EXPR$0=[$1]) LogicalAggregate(group=[{0}], EXPR$0=[MAX($1)]) LogicalProject(f=[$2], e=[$1]) LogicalFilter(condition=[<($0, 3)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1110,13 +1110,13 @@ Calc(select=[a]) +- Join(joinType=[LeftSemiJoin], where=[(b = EXPR$0)], select=[a, b], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[EXPR$0]]) +- Calc(select=[EXPR$0]) +- GroupAggregate(groupBy=[f], select=[f, MAX(e) AS EXPR$0]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[f, e], where=[(d < 3)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1130,21 +1130,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, $1, { LogicalAggregate(group=[{0}], EXPR$1=[COUNT()]) LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1157,9 +1157,9 @@ Join(joinType=[LeftSemiJoin], where=[((a = d) AND (b = EXPR$1))], select=[a, b, LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), NOT(LIKE($2, _UTF-16LE'abc')), IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1167,10 +1167,10 @@ LogicalProject(d=[$0]) Join(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[((b > 10) AND NOT(LIKE(c, 'abc')))]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1183,19 +1183,19 @@ Join(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], leftInputSpec=[ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IS TRUE(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1211,12 +1211,12 @@ SELECT b FROM l WHERE (CASE WHEN a IN LogicalProject(b=[$1]) +- LogicalFilter(condition=[IN(CASE(IN($0, { LogicalProject(EXPR$0=[1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), 1, 2), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1230,11 +1230,11 @@ Calc(select=[b]) : : +- Join(joinType=[InnerJoin], where=[true], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) : : :- Exchange(distribution=[single]) : : : +- Calc(select=[a, b]) - : : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : : +- Exchange(distribution=[single]) : : +- GroupAggregate(select=[COUNT(*) AS c]) : : +- Exchange(distribution=[single]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) + : : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) : +- Exchange(distribution=[single]) : +- Calc(select=[true AS i]) : +- GroupAggregate(groupBy=[EXPR$0], select=[EXPR$0]) @@ -1243,7 +1243,7 @@ Calc(select=[b]) : +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1258,12 +1258,12 @@ LogicalProject(a=[$0]) LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]) LogicalProject(e=[$1]) LogicalFilter(condition=[>($0, 0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1274,15 +1274,15 @@ Calc(select=[a]) : +- Join(joinType=[LeftOuterJoin], where=[true], select=[a, EXPR$0], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) : :- Exchange(distribution=[single]) : : +- Calc(select=[a]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[single]) : +- GroupAggregate(select=[MAX(e) AS EXPR$0]) : +- Exchange(distribution=[single]) : +- Calc(select=[e], where=[(d > 0)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j]]) +- Calc(select=[j]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1296,15 +1296,15 @@ LogicalProject(b=[$1]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($1, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), >($1, $SCALAR_QUERY({ LogicalProject(EXPR$0=[*(0.5:DECIMAL(2, 1), $0)]) LogicalAggregate(group=[{}], agg#0=[SUM($0)]) LogicalProject(j=[$1]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1316,10 +1316,10 @@ Calc(select=[b]) : +- Join(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) : :- Exchange(distribution=[hash[a]]) : : +- Calc(select=[a, b]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d], where=[(e > 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[single]) +- GroupAggregate(select=[SINGLE_VALUE_RETRACT(EXPR$0) AS $f0]) +- Exchange(distribution=[single]) @@ -1327,7 +1327,7 @@ Calc(select=[b]) +- GroupAggregate(select=[SUM(j) AS $f0]) +- Exchange(distribution=[single]) +- Calc(select=[j], where=[(i < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1340,11 +1340,11 @@ Calc(select=[b]) LogicalProject(s=[$1]) +- LogicalFilter(condition=[AND(>($2, 2), IN($3, { LogicalProject(e=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) +- LogicalAggregate(group=[{0}], s=[SUM($1)], agg#1=[COUNT()], agg#2=[MAX($0)]) +- LogicalProject(b=[$1], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1356,10 +1356,10 @@ Calc(select=[s]) : +- GroupAggregate(groupBy=[b], select=[b, SUM(a) AS s, COUNT(*) AS $f2, MAX(b) AS $f3]) : +- Exchange(distribution=[hash[b]]) : +- Calc(select=[b, a]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1377,13 +1377,13 @@ LogicalProject(c=[$2]) LogicalProject(d=[$0]) LogicalProject(d=[$0], e=[$1], f=[$2], i=[$3]) LogicalJoin(condition=[=($1, $4)], joinType=[full]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0], i0=[CAST($0):BIGINT]) LogicalProject(i=[$0]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1392,16 +1392,16 @@ Calc(select=[c]) +- Join(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, c]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) +- Join(joinType=[FullOuterJoin], where=[(e = i0)], select=[d, e, i0], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[e]]) : +- Calc(select=[d, e]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[i0]]) +- Calc(select=[CAST(i AS BIGINT) AS i0], where=[(i > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1415,21 +1415,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($2, { LogicalProject(f1=[$3]) LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalTableFunctionScan(invocation=[table_func($cor0.f)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1445,9 +1445,9 @@ SELECT * FROM l WHERE LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN(+($0, 10), SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[+($0, 100)], EXPR$1=[SUBSTRING($2, 1, 5)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1456,10 +1456,10 @@ Calc(select=[a, b, c]) +- Join(joinType=[LeftSemiJoin], where=[(($f3 = EXPR$0) AND ($f4 = EXPR$1))], select=[a, b, c, $f3, $f4], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[$f3, $f4]]) : +- Calc(select=[a, b, c, (a + 10) AS $f3, SUBSTRING(c, 1, 5) AS $f4]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[EXPR$0, EXPR$1]]) +- Calc(select=[(d + 100) AS EXPR$0, SUBSTRING(f, 1, 5) AS EXPR$1]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1476,21 +1476,21 @@ SELECT * FROM l WHERE (a, b) IN LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, $1, { LogicalProject(EXPR$0=[MAX($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST)], EXPR$1=[MIN($1) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1503,19 +1503,19 @@ Join(joinType=[LeftSemiJoin], where=[((a = $0) AND (b = $1))], select=[a, b, c], LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1529,9 +1529,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), >($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1539,10 +1539,10 @@ LogicalProject(d=[$0]) Join(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b, c], where=[(b > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d], where=[(e < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1555,9 +1555,9 @@ Join(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], leftInputSpec=[ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[IN(+($0, 1), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1566,10 +1566,10 @@ Calc(select=[a, b, c]) +- Join(joinType=[LeftSemiJoin], where=[($f3 = d)], select=[a, b, c, $f3], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[$f3]]) : +- Calc(select=[a, b, c, (a + 1) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -1587,12 +1587,12 @@ LogicalProject(a=[$0]) LogicalUnion(all=[false]) LogicalProject(e=[$1]) LogicalFilter(condition=[>($0, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) LogicalProject(i=[$0]) LogicalFilter(condition=[<($0, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1601,15 +1601,15 @@ Calc(select=[a]) +- Join(joinType=[LeftSemiJoin], where=[(b = e)], select=[a, b], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[a, b]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- GroupAggregate(groupBy=[e], select=[e]) +- Exchange(distribution=[hash[e]]) +- Union(all=[true], union=[e]) :- Calc(select=[e], where=[(d > 10)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Calc(select=[CAST(i AS BIGINT) AS i], where=[(i < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1622,12 +1622,12 @@ Calc(select=[a]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(EXISTS({ LogicalFilter(condition=[<>($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), EXISTS({ LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1636,16 +1636,16 @@ Join(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], leftInputSpec=[NoU :- Exchange(distribution=[single]) : +- Join(joinType=[LeftSemiJoin], where=[(a <> d)], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) : :- Exchange(distribution=[single]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[single]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[single]) +- Calc(select=[], where=[m IS NOT NULL]) +- GroupAggregate(select=[MIN(i) AS m]) +- Exchange(distribution=[single]) +- Calc(select=[true AS i], where=[(j < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1658,29 +1658,29 @@ Join(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], leftInputSpec=[NoU LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[EXISTS({ LogicalFilter(condition=[AND(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) +LogicalTableScan(table=[[default_catalog, default_database, t]]) }), =($cor0.a, $0))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1693,14 +1693,14 @@ Join(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], leftInputSpec=[ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), EXISTS({ LogicalFilter(condition=[>($0, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }), >=($1, 1), EXISTS({ LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1712,15 +1712,15 @@ Join(joinType=[LeftSemiJoin], where=[true], select=[a, b, c, d, e, f], leftInput : : +- Join(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) : : :- Exchange(distribution=[hash[a]]) : : : +- Calc(select=[a, b, c], where=[(b >= 1)]) -: : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : : +- Exchange(distribution=[hash[d]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) : +- Exchange(distribution=[single]) : +- Calc(select=[true AS $f0], where=[m IS NOT NULL]) : +- GroupAggregate(select=[MIN(i) AS m]) : +- Exchange(distribution=[single]) : +- Calc(select=[true AS i], where=[(i > 50)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) +- Exchange(distribution=[single]) +- Calc(select=[], where=[m IS NOT NULL]) +- GroupAggregate(select=[MIN(i) AS m]) @@ -1740,13 +1740,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) LogicalFilter(condition=[AND(=($cor0.a, $0), <>($2, _UTF-16LE'test'))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1755,13 +1755,13 @@ Join(joinType=[LeftSemiJoin], where=[((b = j) AND (a = i))], select=[a, b, c], l :- Exchange(distribution=[hash[b, a]]) : +- Join(joinType=[LeftSemiJoin], where=[((a = d) AND (c = f))], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) : :- Exchange(distribution=[hash[a, c]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d, f]]) : +- Calc(select=[d, f]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j, i]]) +- Calc(select=[j, i], where=[(k <> 'test')]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1777,26 +1777,26 @@ LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), IN($2, { LogicalProject(k=[$2]) LogicalFilter(condition=[=($cor1.e, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor1]]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1809,13 +1809,13 @@ Join(joinType=[LeftSemiJoin], where=[((a = d) AND (b = e))], select=[a, b, c], l LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) LogicalFilter(condition=[=($2, $cor0.c)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1824,13 +1824,13 @@ Join(joinType=[LeftSemiJoin], where=[((b = j) AND (k = c))], select=[a, b, c], l :- Exchange(distribution=[hash[b, c]]) : +- Join(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) : :- Exchange(distribution=[hash[a]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j, k]]) +- Calc(select=[j, k]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1843,12 +1843,12 @@ Join(joinType=[LeftSemiJoin], where=[((b = j) AND (k = c))], select=[a, b, c], l LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }), IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1857,13 +1857,13 @@ Join(joinType=[LeftSemiJoin], where=[(b = j)], select=[a, b, c], leftInputSpec=[ :- Exchange(distribution=[hash[b]]) : +- Join(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) : :- Exchange(distribution=[hash[a]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j]]) +- Calc(select=[j]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -1878,27 +1878,27 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) LogicalProject(d=[$0]) LogicalFilter(condition=[IN($1, { LogicalProject(j=[$1]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -1911,14 +1911,14 @@ Join(joinType=[LeftSemiJoin], where=[(a = d)], select=[a, b, c], leftInputSpec=[ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), NOT(EXISTS({ LogicalFilter(condition=[AND(=($cor0.b, $1), >(CAST($2):BIGINT, 50))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), >=(CAST($2):BIGINT, 1), NOT(EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))], variablesSet=[[$cor0]]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1930,12 +1930,12 @@ Join(joinType=[LeftAntiJoin], where=[(a = i)], select=[a, b, c, d, e, f], leftIn : : +- Join(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) : : :- Exchange(distribution=[hash[a]]) : : : +- Calc(select=[a, b, c], where=[(CAST(c AS BIGINT) >= 1)]) -: : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : : +- Exchange(distribution=[hash[d]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) : +- Exchange(distribution=[hash[j]]) : +- Calc(select=[j], where=[(CAST(k AS BIGINT) > 50)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) +- Exchange(distribution=[hash[i]]) +- Calc(select=[i], where=[(j < 100)]) +- Reused(reference_id=[1]) @@ -1951,14 +1951,14 @@ Join(joinType=[LeftAntiJoin], where=[(a = i)], select=[a, b, c, d, e, f], leftIn LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5]) +- LogicalFilter(condition=[AND(=($0, $3), NOT(EXISTS({ LogicalFilter(condition=[>($0, 50)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), >=($1, 1), NOT(EXISTS({ LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))]) +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, l]]) + +- LogicalTableScan(table=[[default_catalog, default_database, r]]) ]]> @@ -1970,15 +1970,15 @@ Join(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c, d, e, f], leftInputS : : +- Join(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) : : :- Exchange(distribution=[hash[a]]) : : : +- Calc(select=[a, b, c], where=[(b >= 1)]) -: : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : : +- Exchange(distribution=[hash[d]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) : +- Exchange(distribution=[single]) : +- Calc(select=[m IS NOT NULL AS $f0]) : +- GroupAggregate(select=[MIN(i) AS m]) : +- Exchange(distribution=[single]) : +- Calc(select=[true AS i], where=[(i > 50)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[1]) +: +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) +- Exchange(distribution=[single]) +- Calc(select=[m IS NOT NULL AS $f0]) +- GroupAggregate(select=[MIN(i) AS m]) @@ -2000,27 +2000,27 @@ LogicalProject(d=[$0]) LogicalFilter(condition=[NOT(IN($1, { LogicalProject(j=[$1]) LogicalFilter(condition=[=($0, $cor0.d)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))], variablesSet=[[$cor0]]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2033,19 +2033,19 @@ Join(joinType=[LeftAntiJoin], where=[(a IS NULL OR d IS NULL OR (a = d))], selec LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[<($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2058,9 +2058,9 @@ Join(joinType=[LeftAntiJoin], where=[(a < d)], select=[a, b, c], leftInputSpec=[ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[<($cor0.a, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2069,10 +2069,10 @@ Calc(select=[a, b, c]) +- Join(joinType=[LeftAntiJoin], where=[$f3], select=[a, b, c, $f3], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) : +- Calc(select=[a, b, c, (a < 10) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[single]) +- Calc(select=[]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2085,19 +2085,19 @@ Calc(select=[a, b, c]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2109,9 +2109,9 @@ Join(joinType=[LeftAntiJoin], where=[(a = d)], select=[a, b, c], leftInputSpec=[ ($1, 10), LIKE($2, _UTF-16LE'abc'), NOT(EXISTS({ -LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) +LogicalTableScan(table=[[default_catalog, default_database, r]]) }))))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2120,13 +2120,13 @@ Calc(select=[(a + 10) AS EXPR$0, c]) +- Join(joinType=[LeftSemiJoin], where=[true], select=[a, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) : +- Calc(select=[a, c], where=[((b <= 10) AND NOT(LIKE(c, 'abc')))]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[single]) +- Calc(select=[], where=[m IS NOT NULL]) +- GroupAggregate(select=[MIN(i) AS m]) +- Exchange(distribution=[single]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2138,22 +2138,22 @@ Calc(select=[(a + 10) AS EXPR$0, c]) @@ -2165,9 +2165,9 @@ Join(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c], leftInputSpec=[NoUn ($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2175,13 +2175,13 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl Join(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) : +- Calc(select=[a, b, c], where=[(b > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[single]) +- Calc(select=[m IS NOT NULL AS $f0]) +- GroupAggregate(select=[MIN(i) AS m]) +- Exchange(distribution=[single]) +- Calc(select=[true AS i]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2194,13 +2194,13 @@ Join(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c], leftInputSpec=[NoUn LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(NOT(EXISTS({ LogicalFilter(condition=[=($cor0.a, $0)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })), NOT(IN($0, { LogicalProject(i=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2209,13 +2209,13 @@ Join(joinType=[LeftAntiJoin], where=[((a IS NULL OR i IS NULL OR (a = i)) AND (b :- Exchange(distribution=[hash[b]]) : +- Join(joinType=[LeftAntiJoin], where=[(a = d)], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) : :- Exchange(distribution=[hash[a]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- Calc(select=[d]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) +- Exchange(distribution=[hash[j]]) +- Calc(select=[i, j]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k]) ]]> @@ -2229,9 +2229,9 @@ LogicalProject(EXPR$0=[+($0, 10)], c=[$2]) +- LogicalFilter(condition=[NOT(IN(SUBSTRING($2, 1, 5), { LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)]) LogicalFilter(condition=[=(+($cor0.b, 1), $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2240,10 +2240,10 @@ Calc(select=[(a + 10) AS EXPR$0, c]) +- Join(joinType=[LeftAntiJoin], where=[(($f3 IS NULL OR EXPR$0 IS NULL OR ($f3 = EXPR$0)) AND ($f4 = e))], select=[a, c, $f3, $f4], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[$f4]]) : +- Calc(select=[a, c, SUBSTRING(c, 1, 5) AS $f3, (b + 1) AS $f4]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[SUBSTRING(f, 1, 5) AS EXPR$0, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2256,16 +2256,16 @@ Calc(select=[(a + 10) AS EXPR$0, c]) LogicalProject(b=[$1]) +- LogicalFilter(condition=[NOT(IN(CASE(NOT(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), 1, NOT(IN($0, { LogicalProject(EXPR$0=[CAST($1):INTEGER]) - LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })), 2, 3), { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.c, $2)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2283,12 +2283,12 @@ Calc(select=[b]) : : : :- Exchange(distribution=[hash[a]]) : : : : +- Join(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) : : : : :- Exchange(distribution=[single]) - : : : : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : : : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : : : : +- Exchange(distribution=[single]) : : : : +- GroupAggregate(select=[COUNT(*) AS c, COUNT(i) AS ck]) : : : : +- Exchange(distribution=[single]) : : : : +- Calc(select=[i])(reuse_id=[1]) - : : : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])(reuse_id=[2]) + : : : : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[2]) : : : +- Exchange(distribution=[hash[i]]) : : : +- Calc(select=[i, true AS i0]) : : : +- GroupAggregate(groupBy=[i], select=[i]) @@ -2306,7 +2306,7 @@ Calc(select=[b]) : +- Reused(reference_id=[3]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[d, f]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2320,19 +2320,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, $2, { LogicalProject(d=[$0], f=[$2]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2346,19 +2346,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> e))], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[single]) +- Calc(select=[d, e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2372,19 +2372,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[>($cor0.b, 10)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> 10))], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[single]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2398,19 +2398,19 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[=($cor0.b, $1)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2424,9 +2424,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 1), NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[AND(=($cor0.b, $1), >($0, 10))]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })))], variablesSet=[[$cor0]]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2434,10 +2434,10 @@ LogicalProject(d=[$0]) Join(joinType=[LeftAntiJoin], where=[((a IS NULL OR d IS NULL OR (a = d)) AND (b = e))], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[a, b, c], where=[(b > 1)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[d, e], where=[(d > 10)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2450,19 +2450,19 @@ Join(joinType=[LeftAntiJoin], where=[((a IS NULL OR d IS NULL OR (a = d)) AND (b LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2475,9 +2475,9 @@ Join(joinType=[LeftAntiJoin], where=[(a IS NULL OR d IS NULL OR (a = d))], selec LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(>($1, 10), LIKE($2, _UTF-16LE'abc'), NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2485,10 +2485,10 @@ LogicalProject(d=[$0]) Join(joinType=[LeftAntiJoin], where=[(a IS NULL OR d IS NULL OR (a = d))], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) : +- Calc(select=[a, b, c], where=[((b > 10) AND LIKE(c, 'abc'))]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[single]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2501,15 +2501,15 @@ Join(joinType=[LeftAntiJoin], where=[(a IS NULL OR d IS NULL OR (a = d))], selec LogicalProject(b=[$1]) +- LogicalFilter(condition=[NOT(IN($1, CASE(NOT(IN($0, { LogicalProject(i=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) })), 1, NOT(IN($0, { LogicalProject(j=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]]) + LogicalTableScan(table=[[default_catalog, default_database, t2]]) })), 2, 3), { LogicalProject(e=[$1], d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2528,11 +2528,11 @@ Calc(select=[b]) : : : : +- Join(joinType=[InnerJoin], where=[true], select=[a, b, c, ck], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) : : : : :- Exchange(distribution=[single]) : : : : : +- Calc(select=[a, b]) - : : : : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : : : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) : : : : +- Exchange(distribution=[single]) : : : : +- GroupAggregate(select=[COUNT(*) AS c, COUNT(i) AS ck]) : : : : +- Exchange(distribution=[single]) - : : : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]], fields=[i])(reuse_id=[1]) + : : : : +- TableSourceScan(table=[[default_catalog, default_database, t1]], fields=[i])(reuse_id=[1]) : : : +- Exchange(distribution=[hash[i]]) : : : +- Calc(select=[i, true AS i0]) : : : +- GroupAggregate(groupBy=[i], select=[i]) @@ -2541,7 +2541,7 @@ Calc(select=[b]) : : +- Exchange(distribution=[single]) : : +- GroupAggregate(select=[COUNT(*) AS c, COUNT(j) AS ck]) : : +- Exchange(distribution=[single]) - : : +- LegacyTableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]], fields=[j])(reuse_id=[2]) + : : +- TableSourceScan(table=[[default_catalog, default_database, t2]], fields=[j])(reuse_id=[2]) : +- Exchange(distribution=[hash[j]]) : +- Calc(select=[j, true AS i]) : +- GroupAggregate(groupBy=[j], select=[j]) @@ -2549,7 +2549,7 @@ Calc(select=[b]) : +- Reused(reference_id=[2]) +- Exchange(distribution=[single]) +- Calc(select=[e, d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2562,19 +2562,19 @@ Calc(select=[b]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, $2, { LogicalProject(d=[$0], f=[$2]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2587,19 +2587,19 @@ Join(joinType=[LeftAntiJoin], where=[((a IS NULL OR d IS NULL OR (a = d)) AND (c LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN($0, { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2613,9 +2613,9 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(NOT(IN($0, { LogicalProject(d=[$0]) LogicalFilter(condition=[<($1, 100)]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) })), >($1, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2623,10 +2623,10 @@ LogicalProject(d=[$0]) Join(joinType=[LeftAntiJoin], where=[(a IS NULL OR d IS NULL OR (a = d))], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) : +- Calc(select=[a, b, c], where=[(b > 10)]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[single]) +- Calc(select=[d], where=[(e < 100)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> @@ -2639,9 +2639,9 @@ Join(joinType=[LeftAntiJoin], where=[(a IS NULL OR d IS NULL OR (a = d))], selec LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[NOT(IN(*($0, $1), { LogicalProject(d=[$0]) - LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) + LogicalTableScan(table=[[default_catalog, default_database, r]]) }))]) - +- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, l]]) ]]> @@ -2650,10 +2650,10 @@ Calc(select=[a, b, c]) +- Join(joinType=[LeftAntiJoin], where=[($f3 IS NULL OR d IS NULL OR ($f3 = d))], select=[a, b, c, $f3], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) : +- Calc(select=[a, b, c, (a * b) AS $f3]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[single]) +- Calc(select=[d]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/AggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/AggregateTest.xml index d61bf0939352a..7d0572181e755 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/AggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/AggregateTest.xml @@ -21,7 +21,7 @@ limitations under the License. @@ -29,7 +29,7 @@ LogicalProject(b=[AS($0, _UTF-16LE'b')], x=[AS($1.f0, _UTF-16LE'x')], y=[AS($1.f Calc(select=[b, TMP_0.f0 AS x, TMP_0.f1 AS y]) +- GroupAggregate(groupBy=[b], select=[b, CountMinMax(a) AS TMP_0]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -38,14 +38,14 @@ Calc(select=[b, TMP_0.f0 AS x, TMP_0.f1 AS y]) @@ -86,7 +86,7 @@ GroupWindowAggregate(groupBy=[a], window=[SessionGroupWindow('w, rowtime, 900000 @@ -94,7 +94,7 @@ LogicalProject(EXPR$0=[$1]) Calc(select=[EXPR$0]) +- GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS EXPR$0]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c]) ]]> @@ -104,7 +104,7 @@ Calc(select=[EXPR$0]) LogicalProject(b=[$0], EXPR$0=[$1]) +- LogicalAggregate(group=[{0}], EXPR$0=[AVG($1)]) +- LogicalProject(b=[$1], a0=[CAST($0):DOUBLE]) - +- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Table1]]) ]]> @@ -112,7 +112,7 @@ LogicalProject(b=[$0], EXPR$0=[$1]) GroupAggregate(groupBy=[b], select=[b, AVG(a0) AS EXPR$0]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[b, CAST(a AS DOUBLE) AS a0]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c]) ]]> @@ -122,7 +122,7 @@ GroupAggregate(groupBy=[b], select=[b, AVG(a0) AS EXPR$0]) LogicalProject(four=[$1], EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[SUM($2)]) +- LogicalProject(a=[$0], four=[4], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Table1]]) ]]> @@ -131,7 +131,7 @@ Calc(select=[4 AS four, EXPR$0]) +- GroupAggregate(groupBy=[a], select=[a, SUM(b) AS EXPR$0]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c]) ]]> @@ -141,7 +141,7 @@ Calc(select=[4 AS four, EXPR$0]) LogicalProject(four=[$1], EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[SUM($2)]) +- LogicalProject(b=[$1], four=[4], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Table1]]) ]]> @@ -150,7 +150,7 @@ Calc(select=[4 AS four, EXPR$0]) +- GroupAggregate(groupBy=[b], select=[b, SUM(a) AS EXPR$0]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[b, a]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c]) ]]> @@ -159,7 +159,7 @@ Calc(select=[4 AS four, EXPR$0]) @@ -167,7 +167,7 @@ LogicalProject(b=[AS($0, _UTF-16LE'b')], f0=[AS($1.f0, _UTF-16LE'f0')], f1=[AS($ Calc(select=[b, TMP_0.f0 AS f0, TMP_0.f1 AS f1]) +- GroupAggregate(groupBy=[b], select=[b, CountMinMax(a) AS TMP_0]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -177,7 +177,7 @@ Calc(select=[b, TMP_0.f0 AS f0, TMP_0.f1 AS f1]) LogicalProject(EXPR$0=[$1], EXPR$1=[$2]) +- LogicalAggregate(group=[{1}], EXPR$0=[MIN($2)], EXPR$1=[AVG($0)]) +- LogicalProject(a=[$0], d=[MOD($1, 3)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Table1]]) ]]> @@ -186,7 +186,7 @@ Calc(select=[EXPR$0, EXPR$1]) +- GroupAggregate(groupBy=[d], select=[d, MIN(c) AS EXPR$0, AVG(a) AS EXPR$1]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[a, MOD(b, 3) AS d, c]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c]) ]]> @@ -196,7 +196,7 @@ Calc(select=[EXPR$0, EXPR$1]) LogicalFilter(condition=[=($0, 2)]) +- LogicalProject(b=[$0], EXPR$0=[$1]) +- LogicalAggregate(group=[{1}], EXPR$0=[SUM($0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Table1]]) ]]> @@ -205,7 +205,7 @@ Calc(select=[CAST(2 AS INTEGER) AS b, EXPR$0]) +- GroupAggregate(groupBy=[b], select=[b, SUM(a) AS EXPR$0]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[a, b, c], where=[(b = 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c]) ]]> @@ -215,7 +215,7 @@ Calc(select=[CAST(2 AS INTEGER) AS b, EXPR$0]) LogicalProject(EXPR$0=[$1], EXPR$1=[$2]) +- LogicalAggregate(group=[{1}], EXPR$0=[*org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions$WeightedAvg*(DISTINCT $0, $2)], EXPR$1=[*org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions$WeightedAvg*($0, $2)]) +- LogicalProject(a=[$0], c=[$2], b0=[CAST($1):BIGINT]) - +- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Table1]]) ]]> @@ -224,7 +224,7 @@ Calc(select=[EXPR$0, EXPR$1]) +- GroupAggregate(groupBy=[c], select=[c, *org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions$WeightedAvg*(DISTINCT a, b0) AS EXPR$0, *org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions$WeightedAvg*(a, b0) AS EXPR$1]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c, CAST(b AS BIGINT) AS b0]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c]) ]]> @@ -233,7 +233,7 @@ Calc(select=[EXPR$0, EXPR$1]) @@ -241,7 +241,7 @@ LogicalProject(EXPR$0=[$1], EXPR$1=[$2]) Calc(select=[EXPR$0, EXPR$1]) +- GroupAggregate(groupBy=[b], select=[b, SUM(DISTINCT a) AS EXPR$0, COUNT(DISTINCT c) AS EXPR$1]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c]) ]]> @@ -266,7 +266,7 @@ GroupWindowAggregate(window=[SlidingGroupWindow('w, rowtime, 3600000, 900000)], @@ -274,7 +274,7 @@ LogicalProject(b=[AS($0, _UTF-16LE'b')], f0=[AS($1.f0, _UTF-16LE'f0')], f1=[AS($ Calc(select=[b, TMP_0.f0 AS f0, TMP_0.f1 AS f1, TMP_0.f2 AS f2]) +- GroupAggregate(groupBy=[b], select=[b, CountMinMax(a) AS TMP_0]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CalcTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CalcTest.xml index 60f5f30a14254..106ab5becae84 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CalcTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CalcTest.xml @@ -20,13 +20,13 @@ limitations under the License. @@ -34,13 +34,13 @@ Calc(select=[a, b, c, CONCAT(c, '_kid_last') AS kid, (a + 2) AS _c4, b AS b2, 'l @@ -49,13 +49,13 @@ Calc(select=[c]) @@ -63,13 +63,13 @@ Calc(select=[a, b, CAST('xx' AS VARCHAR(2147483647)) AS c], where=[(SEARCH(b, Sa @@ -80,13 +80,13 @@ LogicalFilter(condition=[=(MOD($0, 2), 1)]) +- LogicalFilter(condition=[<($1, 2)]) +- LogicalFilter(condition=[>($0, 0)]) +- LogicalProject(a=[$0], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 0) AND (b < 2) AND (MOD(a, 2) = 1))]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) ]]> @@ -94,13 +94,13 @@ Calc(select=[a, b], where=[((a > 0) AND (b < 2) AND (MOD(a, 2) = 1))]) @@ -109,13 +109,13 @@ Calc(select=[*org.apache.flink.table.planner.expressions.utils.Func24$$2da7dcb3c ($2, _UTF-16LE'xx'))]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 'xx'))]) -+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -137,13 +137,13 @@ Calc(select=[(my_row = ROW(1, 'str')) AS _c0]) @@ -151,13 +151,13 @@ Calc(select=[*org.apache.flink.table.planner.expressions.utils.Func1$$879c853756 diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/ColumnFunctionsTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/ColumnFunctionsTest.xml index 7c9102940a647..86b4902e685c5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/ColumnFunctionsTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/ColumnFunctionsTest.xml @@ -20,13 +20,13 @@ limitations under the License. @@ -34,13 +34,13 @@ Calc(select=[a, b, c, TestFunc(a, b) AS d]) @@ -48,13 +48,13 @@ Calc(select=[CONCAT(string1, string2) AS _c0]) @@ -62,13 +62,13 @@ Calc(select=[b, c, a, e, f, d]) @@ -76,13 +76,13 @@ Calc(select=[a, b, c, f]) @@ -90,13 +90,13 @@ Calc(select=[c]) @@ -105,14 +105,14 @@ Calc(select=[int, long, string1, string2], where=[(CONCAT(string1, string2) = 'a @@ -120,13 +120,13 @@ GroupAggregate(groupBy=[a, b], select=[a, b, COUNT(c) AS EXPR$0]) @@ -134,17 +134,17 @@ Calc(select=[e, f]) @@ -152,14 +152,14 @@ Join(joinType=[InnerJoin], where=[(int1 = int2)], select=[int1, long1, string1, @@ -184,13 +184,13 @@ Calc(select=[c, w0$o0 AS mycount, w0$o1 AS wAvg, w0$o2 AS countDist]) @@ -198,13 +198,13 @@ Calc(select=[a AS d, b]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.xml index 2542e8ddb26fa..7e8d772732953 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.xml @@ -20,14 +20,14 @@ limitations under the License. @@ -42,14 +42,14 @@ FROM MyTable, LATERAL TABLE(func1(c)) AS T * NOT NULL]) ]]> @@ -62,7 +62,7 @@ LogicalProject(c=[$0], d=[$1]) +- LogicalProject(c=[$2], d=[$3], e=[$4]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)]) ]]> @@ -70,7 +70,7 @@ LogicalProject(c=[$0], d=[$1]) ($1, 20)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -83,7 +83,7 @@ LogicalProject(c=[$0], d=[$1]) +- LogicalProject(c=[$2], d=[$3], e=[$4]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)]) ]]> @@ -91,7 +91,7 @@ LogicalProject(c=[$0], d=[$1]) ($1, 20)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -100,7 +100,7 @@ Calc(select=[c, d]) @@ -108,7 +108,7 @@ LogicalProject(c=[$2], s=[$3]) @@ -117,7 +117,7 @@ Calc(select=[c, s]) @@ -125,7 +125,7 @@ LogicalProject(c=[$2], s=[$3]) @@ -134,7 +134,7 @@ Calc(select=[c, s]) @@ -142,7 +142,7 @@ LogicalProject(c=[$2], name=[$3], len=[$4]) @@ -152,7 +152,7 @@ Calc(select=[c, name, len]) LogicalFilter(condition=[>($2, 2)]) +- LogicalProject(c=[$2], name=[$3], len=[$4]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], rowType=[RecordType(VARCHAR(2147483647) name, INTEGER len)]) ]]> @@ -160,7 +160,7 @@ LogicalFilter(condition=[>($2, 2)]) ($1, 2)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -169,7 +169,7 @@ Calc(select=[c, name, len]) @@ -177,7 +177,7 @@ LogicalProject(f0=[AS($3, _UTF-16LE'f0')], f1=[AS($4, _UTF-16LE'f1')]) @@ -192,14 +192,14 @@ FROM MyTable, LATERAL TABLE(str_split('Jack,John')) AS T @@ -207,14 +207,14 @@ Correlate(invocation=[str_split(_UTF-16LE'Jack,John')], correlate=[table(str_spl @@ -229,14 +229,14 @@ FROM MyTable, LATERAL TABLE(str_split('Jack,John', ',')) AS T @@ -251,14 +251,14 @@ FROM MyTable Left JOIN LATERAL TABLE(str_split('Jack,John')) AS T ON TRUE @@ -273,14 +273,14 @@ FROM MyTable Left JOIN LATERAL TABLE(str_split('Jack,John', ',')) AS T ON TRUE @@ -289,7 +289,7 @@ Correlate(invocation=[str_split(_UTF-16LE'Jack,John', _UTF-16LE',')], correlate= @@ -297,7 +297,7 @@ LogicalProject(c=[$2], s=[$3]) @@ -305,14 +305,14 @@ Calc(select=[c, s]) @@ -320,14 +320,14 @@ Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*(SUBSTRIN diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/GroupWindowTableAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/GroupWindowTableAggregateTest.xml index 8db3afd43b1a8..45571d25214aa 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/GroupWindowTableAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/GroupWindowTableAggregateTest.xml @@ -23,7 +23,9 @@ LogicalProject(EXPR$0=[$2], f1=[$1]) +- LogicalWindowTableAggregate(group=[{}], tableAggregate=[[EmptyTableAggFunc($2)]], window=[SlidingGroupWindow('w2, proctime, 20, 10)], properties=[EXPR$0]) +- LogicalProject(proctime=[AS($3, _UTF-16LE'proctime')], c=[$0], f0=[$1], f1=[AS(+($2, 1), _UTF-16LE'f1')]) +- LogicalWindowTableAggregate(group=[{2}], tableAggregate=[[EmptyTableAggFunc($0, $1)]], window=[TumblingGroupWindow('w1, e, 50)], properties=[EXPR$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalWatermarkAssigner(rowtime=[d], watermark=[$3]) + +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[PROCTIME()]) + +- LogicalTableScan(table=[[default_catalog, default_database, Table1]]) ]]> @@ -34,7 +36,9 @@ Calc(select=[EXPR$0, f1]) +- Calc(select=[EXPR$0 AS proctime, c, f0, (f1 + 1) AS f1]) +- GroupWindowTableAggregate(groupBy=[c], window=[TumblingGroupWindow('w1, e, 50)], properties=[EXPR$0], select=[c, EmptyTableAggFunc(a, b) AS (f0, f1), proctime('w1) AS EXPR$0]) +- Exchange(distribution=[hash[c]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- WatermarkAssigner(rowtime=[d], watermark=[d]) + +- Calc(select=[a, b, c, d, PROCTIME() AS e]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c, d]) ]]> @@ -43,7 +47,9 @@ Calc(select=[EXPR$0, f1]) @@ -52,7 +58,9 @@ Calc(select=[f0, (f1 + 1) AS _c1, EXPR$0, EXPR$1]) +- GroupWindowTableAggregate(groupBy=[e], window=[TumblingGroupWindow('w, d, 5)], properties=[EXPR$0, EXPR$1], select=[e, EmptyTableAggFunc(a, b) AS (f0, f1), start('w) AS EXPR$0, end('w) AS EXPR$1]) +- Exchange(distribution=[hash[e]]) +- Calc(select=[a, b, c, d, PROCTIME_MATERIALIZE(e) AS e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- WatermarkAssigner(rowtime=[d], watermark=[d]) + +- Calc(select=[a, b, c, d, PROCTIME() AS e]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c, d]) ]]> @@ -61,7 +69,9 @@ Calc(select=[f0, (f1 + 1) AS _c1, EXPR$0, EXPR$1]) @@ -69,7 +79,9 @@ LogicalProject(f0=[$1], _c1=[AS(+($2, 1), _UTF-16LE'_c1')], EXPR$0=[$3], EXPR$1= Calc(select=[f0, (f1 + 1) AS _c1, EXPR$0, EXPR$1]) +- GroupWindowTableAggregate(groupBy=[c], window=[TumblingGroupWindow('w, d, 5)], properties=[EXPR$0, EXPR$1], select=[c, EmptyTableAggFunc(a, b) AS (f0, f1), start('w) AS EXPR$0, end('w) AS EXPR$1]) +- Exchange(distribution=[hash[c]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- WatermarkAssigner(rowtime=[d], watermark=[d]) + +- Calc(select=[a, b, c, d, PROCTIME() AS e]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c, d]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/GroupWindowTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/GroupWindowTest.xml index f2e645c014159..8e87a1daaf499 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/GroupWindowTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/GroupWindowTest.xml @@ -471,10 +471,12 @@ Calc(select=[string, (EXPR$0 + 1) AS s1, (EXPR$0 + 3) AS s2, EXPR$1 AS x, EXPR$1 LogicalUnion(all=[true]) :- LogicalProject(_c0=[AS(1, _UTF-16LE'_c0')]) : +- LogicalWindowTableAggregate(group=[{}], tableAggregate=[[EmptyTableAggFunc($1, $2)]], window=[SlidingGroupWindow('w1, ts, 3600000, 3600000)], properties=[]) -: +- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(ts, a, b)]]]) +: +- LogicalWatermarkAssigner(rowtime=[ts], watermark=[$0]) +: +- LogicalTableScan(table=[[default_catalog, default_database, Table1]]) +- LogicalProject(_c0=[AS(1, _UTF-16LE'_c0')]) +- LogicalWindowTableAggregate(group=[{}], tableAggregate=[[EmptyTableAggFunc($1, $2)]], window=[SlidingGroupWindow('w1, ts, 7200000, 3600000)], properties=[]) - +- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(ts, a, b)]]]) + +- LogicalWatermarkAssigner(rowtime=[ts], watermark=[$0]) + +- LogicalTableScan(table=[[default_catalog, default_database, Table1]]) ]]> @@ -483,7 +485,8 @@ Union(all=[true], union=[_c0]) :- Calc(select=[1 AS _c0]) : +- GroupWindowTableAggregate(window=[SlidingGroupWindow('w1, ts, 3600000, 3600000)], select=[EmptyTableAggFunc(a, b) AS (f0, f1)]) : +- Exchange(distribution=[single])(reuse_id=[1]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(ts, a, b)]]], fields=[ts, a, b]) +: +- WatermarkAssigner(rowtime=[ts], watermark=[ts]) +: +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[ts, a, b]) +- Calc(select=[1 AS _c0]) +- GroupWindowTableAggregate(window=[SlidingGroupWindow('w1, ts, 7200000, 3600000)], select=[EmptyTableAggFunc(a, b) AS (f0, f1)]) +- Reused(reference_id=[1]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonAggregateTest.xml index 21403890fb700..015e4fba33994 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonAggregateTest.xml @@ -21,14 +21,14 @@ limitations under the License. @@ -37,14 +37,14 @@ PythonGroupAggregate(groupBy=[b], select=[b, TestPythonAggregateFunction(a, c) A @@ -53,14 +53,14 @@ PythonGroupAggregate(groupBy=[b], select=[b, TestPythonAggregateFunction(a, c) A diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonCalcTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonCalcTest.xml index 3214c45b9446a..b77ddaef7a60e 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonCalcTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonCalcTest.xml @@ -23,14 +23,14 @@ limitations under the License. diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonGroupWindowAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonGroupWindowAggregateTest.xml index f249b84bfaefc..e4941a3eb0328 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonGroupWindowAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonGroupWindowAggregateTest.xml @@ -21,14 +21,15 @@ limitations under the License. @@ -37,7 +38,8 @@ PythonGroupWindowAggregate(groupBy=[b], window=[SlidingGroupWindow('w, proctime, @@ -45,7 +47,8 @@ LogicalProject(b=[$0], EXPR$0=[$2], EXPR$1=[$3], EXPR$2=[$1]) Calc(select=[b, EXPR$0, EXPR$1, EXPR$2]) +- PythonGroupWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w, rowtime, 5)], properties=[EXPR$0, EXPR$1], select=[b, PandasAggregateFunction(a, c) AS EXPR$2, start('w) AS EXPR$0, end('w) AS EXPR$1]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, rowtime)]]], fields=[a, b, c, rowtime]) + +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, rowtime]) ]]> @@ -54,7 +57,8 @@ Calc(select=[b, EXPR$0, EXPR$1, EXPR$2]) @@ -62,7 +66,8 @@ LogicalProject(b=[$0], EXPR$0=[$2], EXPR$1=[$3], EXPR$2=[$1]) Calc(select=[b, EXPR$0, EXPR$1, EXPR$2]) +- PythonGroupWindowAggregate(groupBy=[b], window=[SlidingGroupWindow('w, rowtime, 5, 2)], properties=[EXPR$0, EXPR$1], select=[b, PandasAggregateFunction(a, c) AS EXPR$2, start('w) AS EXPR$0, end('w) AS EXPR$1]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, rowtime)]]], fields=[a, b, c, rowtime]) + +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, rowtime]) ]]> @@ -71,14 +76,15 @@ Calc(select=[b, EXPR$0, EXPR$1, EXPR$2]) @@ -87,14 +93,15 @@ PythonGroupWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w, proctime @@ -103,7 +110,8 @@ PythonGroupWindowAggregate(groupBy=[b], window=[SlidingGroupWindow('w, proctime, @@ -111,7 +119,8 @@ LogicalProject(b=[$0], EXPR$0=[$2], EXPR$1=[$3], EXPR$2=[$1]) Calc(select=[b, EXPR$0, EXPR$1, EXPR$2]) +- PythonGroupWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w, rowtime, 5)], properties=[EXPR$0, EXPR$1], select=[b, TestPythonAggregateFunction(a, c) AS EXPR$2, start('w) AS EXPR$0, end('w) AS EXPR$1]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, rowtime)]]], fields=[a, b, c, rowtime]) + +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, rowtime]) ]]> @@ -120,7 +129,8 @@ Calc(select=[b, EXPR$0, EXPR$1, EXPR$2]) @@ -128,7 +138,8 @@ LogicalProject(b=[$0], EXPR$0=[$2], EXPR$1=[$3], EXPR$2=[$1]) Calc(select=[b, EXPR$0, EXPR$1, EXPR$2]) +- PythonGroupWindowAggregate(groupBy=[b], window=[SlidingGroupWindow('w, rowtime, 5, 2)], properties=[EXPR$0, EXPR$1], select=[b, TestPythonAggregateFunction(a, c) AS EXPR$2, start('w) AS EXPR$0, end('w) AS EXPR$1]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, rowtime)]]], fields=[a, b, c, rowtime]) + +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, rowtime]) ]]> @@ -137,14 +148,15 @@ Calc(select=[b, EXPR$0, EXPR$1, EXPR$2]) @@ -153,7 +165,8 @@ PythonGroupWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w, proctime @@ -161,7 +174,8 @@ LogicalProject(b=[$0], EXPR$0=[$2], EXPR$1=[$3], EXPR$2=[$1]) Calc(select=[b, EXPR$0, EXPR$1, EXPR$2]) +- PythonGroupWindowAggregate(groupBy=[b], window=[SessionGroupWindow('w, rowtime, 7)], properties=[EXPR$0, EXPR$1], select=[b, TestPythonAggregateFunction(a, c) AS EXPR$2, start('w) AS EXPR$0, end('w) AS EXPR$1]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, rowtime)]]], fields=[a, b, c, rowtime]) + +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, rowtime]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonOverWindowAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonOverWindowAggregateTest.xml index 8884f86b6b47d..296dd7cbbf8f8 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonOverWindowAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonOverWindowAggregateTest.xml @@ -19,32 +19,34 @@ limitations under the License. @@ -52,7 +54,8 @@ Calc(select=[b, w0$o0 AS _c1]) @@ -60,7 +63,8 @@ LogicalProject(b=[$1], _c1=[AS(*org.apache.flink.table.planner.runtime.utils.Jav Calc(select=[b, w0$o0 AS _c1]) +- PythonOverAggregate(partitionBy=[b], orderBy=[rowtime ASC], window=[ RANG BETWEEN 10000 PRECEDING AND CURRENT ROW], select=[a, b, c, rowtime, PandasAggregateFunction(a, c) AS w0$o0]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, rowtime)]]], fields=[a, b, c, rowtime]) + +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, rowtime]) ]]> @@ -68,7 +72,8 @@ Calc(select=[b, w0$o0 AS _c1]) @@ -76,7 +81,8 @@ LogicalProject(b=[$1], _c1=[AS(*org.apache.flink.table.planner.runtime.utils.Jav Calc(select=[b, w0$o0 AS _c1]) +- PythonOverAggregate(partitionBy=[b], orderBy=[rowtime ASC], window=[ ROWS BETWEEN 10 PRECEDING AND CURRENT ROW], select=[a, b, c, rowtime, PandasAggregateFunction(a, c) AS w0$o0]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, rowtime)]]], fields=[a, b, c, rowtime]) + +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, rowtime]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonTableAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonTableAggregateTest.xml index 96b2bf7b878dd..9f6abdedad3cc 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonTableAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonTableAggregateTest.xml @@ -21,7 +21,7 @@ limitations under the License. @@ -29,7 +29,7 @@ LogicalProject(d=[AS($1, _UTF-16LE'd')], e=[AS($2, _UTF-16LE'e')]) Calc(select=[f0 AS d, f1 AS e]) +- PythonGroupTableAggregate(groupBy=[b], select=[b, PythonEmptyTableAggFunc(a, c) AS (f0, f1)]) +- Exchange(distribution=[hash[b]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -38,7 +38,7 @@ Calc(select=[f0 AS d, f1 AS e]) @@ -46,7 +46,7 @@ LogicalProject(d=[AS($0, _UTF-16LE'd')], e=[AS($1, _UTF-16LE'e')]) Calc(select=[f0 AS d, f1 AS e]) +- PythonGroupTableAggregate(select=[PythonEmptyTableAggFunc(a, c) AS (f0, f1)]) +- Exchange(distribution=[single]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> @@ -56,7 +56,7 @@ Calc(select=[f0 AS d, f1 AS e]) LogicalProject(d=[AS($1, _UTF-16LE'd')], e=[AS($2, _UTF-16LE'e')]) +- LogicalTableAggregate(group=[{1}], tableAggregate=[[PythonEmptyTableAggFunc($0, $2)]]) +- LogicalProject(a=[$0], b=[$1], $f3=[+($2, 1)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -65,7 +65,7 @@ Calc(select=[f0 AS d, f1 AS e]) +- PythonGroupTableAggregate(groupBy=[b], select=[b, PythonEmptyTableAggFunc(a, $f3) AS (f0, f1)]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[a, b, (c + 1) AS $f3]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/SetOperatorsTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/SetOperatorsTest.xml index 58e852bba7a70..c87dac22aa4a9 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/SetOperatorsTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/SetOperatorsTest.xml @@ -21,19 +21,19 @@ limitations under the License. @@ -44,8 +44,8 @@ LogicalProject(a=[$1], b=[$0], c=[$2]) +- LogicalAggregate(group=[{1}], EXPR$0=[SUM($0)], EXPR$1=[COUNT($2)]) +- LogicalFilter(condition=[>($0, 0)]) +- LogicalUnion(all=[true]) - :- LogicalTableScan(table=[[default_catalog, default_database, left, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, right, source: [TestTableSource(a, b, c)]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, left]]) + +- LogicalTableScan(table=[[default_catalog, default_database, right]]) ]]> @@ -55,9 +55,9 @@ Calc(select=[EXPR$0 AS a, b, EXPR$1 AS c]) +- Exchange(distribution=[hash[b]]) +- Union(all=[true], union=[a, b, c]) :- Calc(select=[a, b, c], where=[(a > 0)]) - : +- LegacyTableSourceScan(table=[[default_catalog, default_database, left, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- TableSourceScan(table=[[default_catalog, default_database, left]], fields=[a, b, c]) +- Calc(select=[a, b, c], where=[(a > 0)]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, right, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, right]], fields=[a, b, c]) ]]> @@ -68,22 +68,22 @@ LogicalFilter(condition=[IN($0, { LogicalProject(EXPR$0=[$1]) LogicalAggregate(group=[{1}], EXPR$0=[SUM($0)]) LogicalFilter(condition=[LIKE($1, _UTF-16LE'%Hanoi%')]) - LogicalTableScan(table=[[default_catalog, default_database, tableB, source: [TestTableSource(x, y)]]]) + LogicalTableScan(table=[[default_catalog, default_database, tableB]]) })]) -+- LogicalTableScan(table=[[default_catalog, default_database, tableA, source: [TestTableSource(a, b, c)]]]) ++- LogicalTableScan(table=[[default_catalog, default_database, tableA]]) ]]> @@ -93,18 +93,18 @@ Join(joinType=[LeftSemiJoin], where=[(a = EXPR$0)], select=[a, b, c], leftInputS LogicalProject(b=[$1], c=[$2]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalTableScan(table=[[default_catalog, default_database, left, source: [TestTableSource(a, b, c)]]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, left]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, right, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, right]]) ]]> @@ -113,12 +113,12 @@ Union(all=[true], union=[b, c]) @@ -127,13 +127,13 @@ Join(joinType=[LeftSemiJoin], where=[(b = w)], select=[a, b, c], leftInputSpec=[ :- Exchange(distribution=[hash[b]]) : +- Join(joinType=[LeftSemiJoin], where=[(a = x)], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) : :- Exchange(distribution=[hash[a]]) -: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, tableA, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, tableA]], fields=[a, b, c]) : +- Exchange(distribution=[hash[x]]) : +- Calc(select=[x]) -: +- LegacyTableSourceScan(table=[[default_catalog, default_database, tableB, source: [TestTableSource(x, y)]]], fields=[x, y]) +: +- TableSourceScan(table=[[default_catalog, default_database, tableB]], fields=[x, y]) +- Exchange(distribution=[hash[w]]) +- Calc(select=[w]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, tableC, source: [TestTableSource(w, z)]]], fields=[w, z]) + +- TableSourceScan(table=[[default_catalog, default_database, tableC]], fields=[w, z]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TableAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TableAggregateTest.xml index 5fbf36cd769e0..94e19398cd677 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TableAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TableAggregateTest.xml @@ -21,14 +21,14 @@ limitations under the License. @@ -37,7 +37,9 @@ GroupTableAggregate(groupBy=[c], select=[c, EmptyTableAggFunc(a) AS (f0, f1)]) @@ -45,7 +47,9 @@ LogicalProject(a=[AS($0, _UTF-16LE'a')], b=[AS($1, _UTF-16LE'b')]) Calc(select=[f0 AS a, f1 AS b]) +- GroupTableAggregate(select=[*org.apache.flink.table.planner.utils.EmptyTableAggFunc*(b) AS (f0, f1)]) +- Exchange(distribution=[single]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- WatermarkAssigner(rowtime=[d], watermark=[d]) + +- Calc(select=[a, b, c, d, PROCTIME() AS e]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c, d]) ]]> @@ -55,7 +59,9 @@ Calc(select=[f0 AS a, f1 AS b]) LogicalProject(bb=[AS($0, _UTF-16LE'bb')], _c1=[+(AS($1, _UTF-16LE'x'), 1)], y=[AS($2, _UTF-16LE'y')]) +- LogicalTableAggregate(group=[{2}], tableAggregate=[[*org.apache.flink.table.planner.utils.EmptyTableAggFunc*($0, $1)]]) +- LogicalProject(a=[$0], b=[$1], bb=[MOD($1, 5)]) - +- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c, d, e)]]]) + +- LogicalWatermarkAssigner(rowtime=[d], watermark=[$3]) + +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[PROCTIME()]) + +- LogicalTableScan(table=[[default_catalog, default_database, Table1]]) ]]> @@ -64,7 +70,9 @@ Calc(select=[bb, (f0 + 1) AS _c1, f1 AS y]) +- GroupTableAggregate(groupBy=[bb], select=[bb, *org.apache.flink.table.planner.utils.EmptyTableAggFunc*(a, b) AS (f0, f1)]) +- Exchange(distribution=[hash[bb]]) +- Calc(select=[a, b, MOD(b, 5) AS bb]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- WatermarkAssigner(rowtime=[d], watermark=[d]) + +- Calc(select=[a, b, d]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c, d]) ]]> @@ -73,14 +81,18 @@ Calc(select=[bb, (f0 + 1) AS _c1, f1 AS y]) @@ -89,7 +101,9 @@ GroupTableAggregate(groupBy=[f0], select=[f0, EmptyTableAggFuncWithIntResultType @@ -97,7 +111,9 @@ LogicalProject(a=[*org.apache.flink.table.planner.expressions.utils.Func0$$6ad06 Calc(select=[*org.apache.flink.table.planner.expressions.utils.Func0$$6ad060e6c46e5cd996d7b888db472ebc*(f0) AS a, f1 AS b]) +- GroupTableAggregate(select=[EmptyTableAggFunc(a, b) AS (f0, f1)]) +- Exchange(distribution=[single]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- WatermarkAssigner(rowtime=[d], watermark=[d]) + +- Calc(select=[a, b, c, d, PROCTIME() AS e]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c, d]) ]]> @@ -106,7 +122,9 @@ Calc(select=[*org.apache.flink.table.planner.expressions.utils.Func0$$6ad060e6c4 @@ -114,7 +132,9 @@ LogicalProject(a=[$0], b=[$1]) GroupTableAggregate(select=[EmptyTableAggFunc(d, e) AS (f0, f1)]) +- Exchange(distribution=[single]) +- Calc(select=[a, b, c, CAST(d AS TIMESTAMP(3)) AS d, PROCTIME_MATERIALIZE(e) AS e]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) + +- WatermarkAssigner(rowtime=[d], watermark=[d]) + +- Calc(select=[a, b, c, d, PROCTIME() AS e]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c, d]) ]]> @@ -123,14 +143,18 @@ GroupTableAggregate(select=[EmptyTableAggFunc(d, e) AS (f0, f1)]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.xml index 33c92d4f41c41..02dd026a67da8 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.xml @@ -21,7 +21,7 @@ limitations under the License. @@ -31,7 +31,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[hash[b]]) +- LocalGroupAggregate(groupBy=[b], select=[b, COUNT(a) AS count$0]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c]) ]]> @@ -41,7 +41,7 @@ Calc(select=[EXPR$0]) LogicalProject(b=[$0], EXPR$0=[$1]) +- LogicalAggregate(group=[{0}], EXPR$0=[AVG($1)]) +- LogicalProject(b=[$1], a0=[CAST($0):DOUBLE]) - +- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Table1]]) ]]> @@ -51,7 +51,7 @@ GlobalGroupAggregate(groupBy=[b], select=[b, AVG((sum$0, count$1)) AS EXPR$0]) +- LocalGroupAggregate(groupBy=[b], select=[b, AVG(a0) AS (sum$0, count$1)]) +- Calc(select=[b, CAST(a AS DOUBLE) AS a0]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c]) ]]> @@ -61,7 +61,7 @@ GlobalGroupAggregate(groupBy=[b], select=[b, AVG((sum$0, count$1)) AS EXPR$0]) LogicalProject(EXPR$0=[$1], EXPR$1=[$2]) +- LogicalAggregate(group=[{1}], EXPR$0=[MIN($2)], EXPR$1=[AVG($0)]) +- LogicalProject(a=[$0], d=[MOD($1, 3)], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Table1]]) ]]> @@ -72,7 +72,7 @@ Calc(select=[EXPR$0, EXPR$1]) +- LocalGroupAggregate(groupBy=[d], select=[d, MIN(c) AS min$0, AVG(a) AS (sum$1, count$2)]) +- Calc(select=[a, MOD(b, 3) AS d, c]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c]) ]]> @@ -82,7 +82,7 @@ Calc(select=[EXPR$0, EXPR$1]) LogicalProject(four=[$1], EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[SUM($2)]) +- LogicalProject(a=[$0], four=[4], b=[$1]) - +- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Table1]]) ]]> @@ -93,7 +93,7 @@ Calc(select=[4 AS four, EXPR$0]) +- LocalGroupAggregate(groupBy=[a], select=[a, SUM(b) AS sum$0]) +- Calc(select=[a, b]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c]) ]]> @@ -103,7 +103,7 @@ Calc(select=[4 AS four, EXPR$0]) LogicalProject(four=[$1], EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[SUM($2)]) +- LogicalProject(b=[$1], four=[4], a=[$0]) - +- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Table1]]) ]]> @@ -114,7 +114,7 @@ Calc(select=[4 AS four, EXPR$0]) +- LocalGroupAggregate(groupBy=[b], select=[b, SUM(a) AS sum$0]) +- Calc(select=[b, a]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c]) ]]> @@ -124,7 +124,7 @@ Calc(select=[4 AS four, EXPR$0]) LogicalFilter(condition=[=($0, 2)]) +- LogicalProject(b=[$0], EXPR$0=[$1]) +- LogicalAggregate(group=[{1}], EXPR$0=[SUM($0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Table1]]) ]]> @@ -135,7 +135,7 @@ Calc(select=[CAST(2 AS INTEGER) AS b, EXPR$0]) +- LocalGroupAggregate(groupBy=[b], select=[b, SUM(a) AS sum$0]) +- Calc(select=[a, b, c], where=[(b = 2)]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/validation/AggregateValidationTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/validation/AggregateValidationTest.xml index 5db02638d2a7f..ad02d6eafc77c 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/validation/AggregateValidationTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/validation/AggregateValidationTest.xml @@ -21,7 +21,7 @@ limitations under the License. @@ -29,7 +29,7 @@ LogicalProject(_c0=[func(_UTF-16LE'abc')]) Calc(select=[func('abc') AS _c0]) +- GroupAggregate(groupBy=[a], select=[a]) +- Exchange(distribution=[hash[a]]) - +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/CalcTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/CalcTest.scala index 7d63dea41d7a9..8aa4255cc1b27 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/CalcTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/CalcTest.scala @@ -17,13 +17,12 @@ */ package org.apache.flink.table.planner.plan.batch.sql -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.java.typeutils.TypeExtractor import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.utils.MyPojo import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.NonDeterministicUdf -import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedTableFunctions.{JavaTableFunc1, StringSplit} +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedTableFunctions.StringSplit import org.apache.flink.table.planner.utils.TableTestBase +import org.apache.flink.table.types.AbstractDataType import org.assertj.core.api.Assertions.assertThatExceptionOfType import org.junit.jupiter.api.{BeforeEach, Test} @@ -158,7 +157,7 @@ class CalcTest extends TableTestBase { def testPojoType(): Unit = { util.addTableSource( "MyTable4", - Array[TypeInformation[_]](TypeExtractor.createTypeInfo(classOf[MyPojo])), + Array[AbstractDataType[_]](DataTypes.STRUCTURED(classOf[MyPojo])), Array("a")) util.verifyExecPlan("SELECT a FROM MyTable4") } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala index 2a2ce36e9832e..73f3ac31a2bc6 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala @@ -17,12 +17,11 @@ */ package org.apache.flink.table.planner.plan.batch.sql -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.java.typeutils.GenericTypeInfo import org.apache.flink.table.api._ import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.planner.plan.utils.NonPojo import org.apache.flink.table.planner.utils.TableTestBase +import org.apache.flink.table.types.AbstractDataType import org.assertj.core.api.Assertions.assertThatExceptionOfType import org.junit.jupiter.api.{BeforeEach, Test} @@ -127,10 +126,11 @@ class SetOperatorsTest extends TableTestBase { val util = batchTestUtil() util.addTableSource( "A", - Array[TypeInformation[_]]( - new GenericTypeInfo(classOf[NonPojo]), - new GenericTypeInfo(classOf[NonPojo])), - Array("a", "b")) + Array[AbstractDataType[_]]( + DataTypes.STRUCTURED(classOf[NonPojo]), + DataTypes.STRUCTURED(classOf[NonPojo])), + Array("a", "b") + ) util.verifyExecPlan("SELECT a FROM A UNION ALL SELECT b FROM A") } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.scala index 8360bafdca7d8..b5ae83c3d5663 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.scala @@ -60,12 +60,6 @@ class TableScanTest extends TableTestBase { """.stripMargin) } - @Test - def testLegacyTableSourceScan(): Unit = { - util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) - util.verifyExecPlan("SELECT * FROM MyTable") - } - @Test def testDDLTableScan(): Unit = { util.addTable(""" diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/WindowTableFunctionTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/WindowTableFunctionTest.scala index 5472f0e40e7eb..10eeb3b835143 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/WindowTableFunctionTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/WindowTableFunctionTest.scala @@ -39,7 +39,8 @@ class WindowTableFunctionTest extends TableTestBase { | b bigint, | c as proctime() |) with ( - | 'connector' = 'COLLECTION' + | 'connector' = 'values', + | 'bounded' = 'true' |) |""".stripMargin) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/AggregateTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/AggregateTestBase.scala index ee12cade68699..d34a7c36b8468 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/AggregateTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/AggregateTestBase.scala @@ -17,12 +17,10 @@ */ package org.apache.flink.table.planner.plan.batch.sql.agg -import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.api._ -import org.apache.flink.table.legacy.api.Types import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.{VarSum1AggFunction, VarSum2AggFunction} import org.apache.flink.table.planner.utils.{BatchTableTestUtil, TableTestBase} -import org.apache.flink.table.runtime.typeutils.DecimalDataTypeInfo +import org.apache.flink.table.types.AbstractDataType import org.assertj.core.api.Assertions.assertThatThrownBy import org.junit.jupiter.api.TestTemplate @@ -32,20 +30,20 @@ abstract class AggregateTestBase extends TableTestBase { protected val util: BatchTableTestUtil = batchTestUtil() util.addTableSource( "MyTable", - Array[TypeInformation[_]]( - Types.BYTE, - Types.SHORT, - Types.INT, - Types.LONG, - Types.FLOAT, - Types.DOUBLE, - Types.BOOLEAN, - Types.STRING, - Types.LOCAL_DATE, - Types.LOCAL_TIME, - Types.LOCAL_DATE_TIME, - DecimalDataTypeInfo.of(30, 20), - DecimalDataTypeInfo.of(10, 5) + Array[AbstractDataType[_]]( + DataTypes.TINYINT(), + DataTypes.SMALLINT, + DataTypes.INT, + DataTypes.BIGINT, + DataTypes.FLOAT, + DataTypes.DOUBLE, + DataTypes.BOOLEAN, + DataTypes.STRING, + DataTypes.DATE, + DataTypes.TIME, + DataTypes.TIMESTAMP(3), + DataTypes.DECIMAL(30, 20), + DataTypes.DECIMAL(10, 5) ), Array( "byte", diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.scala index 4ae213b95e4d5..1ec48783dc425 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.scala @@ -17,11 +17,10 @@ */ package org.apache.flink.table.planner.plan.batch.table -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.java.typeutils.GenericTypeInfo import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.utils.NonPojo import org.apache.flink.table.planner.utils.TableTestBase +import org.apache.flink.table.types.AbstractDataType import org.junit.jupiter.api.Test @@ -66,10 +65,11 @@ class SetOperatorsTest extends TableTestBase { val util = batchTestUtil() val t = util.addTableSource( "A", - Array[TypeInformation[_]]( - new GenericTypeInfo(classOf[NonPojo]), - new GenericTypeInfo(classOf[NonPojo])), - Array("a", "b")) + Array[AbstractDataType[_]]( + DataTypes.STRUCTURED(classOf[NonPojo]), + DataTypes.STRUCTURED(classOf[NonPojo])), + Array("a", "b") + ) val in = t.select('a).unionAll(t.select('b)) util.verifyExecPlan(in) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/DistinctAggregateTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/DistinctAggregateTestBase.scala index ee4c5fc69255c..093fc44e6cfa4 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/DistinctAggregateTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/DistinctAggregateTestBase.scala @@ -17,10 +17,9 @@ */ package org.apache.flink.table.planner.plan.common -import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.api._ -import org.apache.flink.table.legacy.api.Types import org.apache.flink.table.planner.utils.{BatchTableTestUtil, TableTestBase} +import org.apache.flink.table.types.AbstractDataType import org.assertj.core.api.Assertions.assertThatExceptionOfType import org.junit.jupiter.api.{BeforeEach, Test} @@ -197,7 +196,7 @@ abstract class DistinctAggregateTestBase(withExecPlan: Boolean) extends TableTes def testTooManyDistinctAggOnDifferentColumn(): Unit = { // max group count must be less than 64 val fieldNames = (0 until 64).map(i => s"f$i").toArray - val fieldTypes: Array[TypeInformation[_]] = Array.fill(fieldNames.length)(Types.INT) + val fieldTypes: Array[AbstractDataType[_]] = Array.fill(fieldNames.length)(DataTypes.INT) util.addTableSource("MyTable64", fieldTypes, fieldNames) val distinctList = fieldNames.map(f => s"COUNT(DISTINCT $f)").mkString(", ") diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/UnnestTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/UnnestTestBase.scala index c47e1ddf0402d..a16b91c7b1225 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/UnnestTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/UnnestTestBase.scala @@ -17,10 +17,9 @@ */ package org.apache.flink.table.planner.plan.common -import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.api._ -import org.apache.flink.table.legacy.api.Types import org.apache.flink.table.planner.utils.{TableTestBase, TableTestUtil} +import org.apache.flink.table.types.AbstractDataType import org.junit.jupiter.api.Test @@ -102,7 +101,10 @@ abstract class UnnestTestBase(withExecPlan: Boolean) extends TableTestBase { def testCrossWithUnnestForMap(): Unit = { util.addTableSource( "MyTable", - Array[TypeInformation[_]](Types.INT, Types.LONG, Types.MAP(Types.STRING, Types.STRING)), + Array[AbstractDataType[_]]( + DataTypes.INT, + DataTypes.BIGINT, + DataTypes.MAP(DataTypes.STRING, DataTypes.STRING)), Array("a", "b", "c")) verifyPlan("SELECT a, b, v FROM MyTable CROSS JOIN UNNEST(c) as f(k, v)") } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/DecomposeGroupingSetsRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/DecomposeGroupingSetsRuleTest.scala index ca41d93a9f675..b44cf661faa30 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/DecomposeGroupingSetsRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/DecomposeGroupingSetsRuleTest.scala @@ -17,11 +17,10 @@ */ package org.apache.flink.table.planner.plan.rules.logical -import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.api._ -import org.apache.flink.table.legacy.api.Types import org.apache.flink.table.planner.plan.optimize.program.FlinkBatchProgram import org.apache.flink.table.planner.utils.TableTestBase +import org.apache.flink.table.types.AbstractDataType import org.assertj.core.api.Assertions.assertThatExceptionOfType import org.junit.jupiter.api.Test @@ -131,7 +130,7 @@ class DecomposeGroupingSetsRuleTest extends TableTestBase { def testTooManyGroupingFields(): Unit = { // max group count must be less than 64 val fieldNames = (0 until 64).map(i => s"f$i").toArray - val fieldTypes: Array[TypeInformation[_]] = Array.fill(fieldNames.length)(Types.INT) + val fieldTypes: Array[AbstractDataType[_]] = Array.fill(fieldNames.length)(DataTypes.INT) util.addTableSource("MyTable64", fieldTypes, fieldNames) val fields = fieldNames.mkString(",") diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.scala index 90991ccdbb780..0e4067715a043 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.scala @@ -17,14 +17,10 @@ */ package org.apache.flink.table.planner.plan.rules.logical -import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.api._ -import org.apache.flink.table.legacy.api.Types import org.apache.flink.table.planner.plan.optimize.program._ -import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.utils.TableTestBase -import com.google.common.collect.ImmutableSet import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.rel.rules._ import org.apache.calcite.tools.RuleSets @@ -71,12 +67,17 @@ class FlinkAggregateJoinTransposeRuleTest extends TableTestBase { util.replaceBatchProgram(program) util.addTableSource[(Int, Int, String)]("T", 'a, 'b, 'c) - util.addTableSource( - "T2", - Array[TypeInformation[_]](Types.INT, Types.INT, Types.STRING), - Array("a2", "b2", "c2"), - FlinkStatistic.builder().uniqueKeys(ImmutableSet.of(ImmutableSet.of("b2"))).build() - ) + util.tableEnv.executeSql(s""" + |CREATE TABLE T2 ( + | a2 INT, + | b2 INT PRIMARY KEY NOT ENFORCED, + | c2 STRING + |) WITH ( + | 'connector' = 'values', + | 'bounded' = 'true', + | 'enable-projection-push-down' = 'false' + |) + |""".stripMargin) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateRemoveRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateRemoveRuleTest.scala index 79ec22f0fd9ec..a443fe84b4886 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateRemoveRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateRemoveRuleTest.scala @@ -17,17 +17,13 @@ */ package org.apache.flink.table.planner.plan.rules.logical -import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.api._ -import org.apache.flink.table.legacy.api.Types import org.apache.flink.table.planner.plan.nodes.FlinkConventions import org.apache.flink.table.planner.plan.nodes.logical._ import org.apache.flink.table.planner.plan.optimize.program._ import org.apache.flink.table.planner.plan.rules.FlinkBatchRuleSets -import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.utils.TableTestBase -import com.google.common.collect.ImmutableSet import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.rel.rules._ import org.apache.calcite.tools.RuleSets @@ -77,7 +73,7 @@ class FlinkAggregateRemoveRuleTest extends TableTestBase { FlinkLogicalJoin.CONVERTER, FlinkLogicalValues.CONVERTER, FlinkLogicalExpand.CONVERTER, - FlinkLogicalLegacyTableSourceScan.CONVERTER, + FlinkLogicalTableSourceScan.CONVERTER, FlinkLogicalLegacySink.CONVERTER )) .setRequiredOutputTraits(Array(FlinkConventions.LOGICAL)) @@ -86,18 +82,29 @@ class FlinkAggregateRemoveRuleTest extends TableTestBase { util.replaceBatchProgram(programs) util.addTableSource[(Int, Int, String)]("MyTable1", 'a, 'b, 'c) - util.addTableSource( - "MyTable2", - Array[TypeInformation[_]](Types.INT, Types.INT, Types.STRING), - Array("a", "b", "c"), - FlinkStatistic.builder().uniqueKeys(ImmutableSet.of(ImmutableSet.of("a"))).build() - ) - util.addTableSource( - "MyTable3", - Array[TypeInformation[_]](Types.INT, Types.INT, Types.STRING, Types.STRING), - Array("a", "b", "c", "d"), - FlinkStatistic.builder().uniqueKeys(ImmutableSet.of(ImmutableSet.of("a"))).build() - ) + util.tableEnv.executeSql(s""" + |CREATE TABLE MyTable2 ( + | a INT PRIMARY KEY NOT ENFORCED, + | b INT, + | c STRING + |) WITH ( + | 'connector' = 'values', + | 'bounded' = 'true', + | 'enable-projection-push-down' = 'false' + |) + |""".stripMargin) + util.tableEnv.executeSql(s""" + |CREATE TABLE MyTable3 ( + | a INT PRIMARY KEY NOT ENFORCED, + | b INT, + | c STRING, + | d STRING + |) WITH ( + | 'connector' = 'values', + | 'bounded' = 'true', + | 'enable-projection-push-down' = 'false' + |) + |""".stripMargin) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRuleTest.scala index 6ee1fa450237a..e86c6dfc9d578 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRuleTest.scala @@ -19,7 +19,7 @@ package org.apache.flink.table.planner.plan.rules.logical import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.nodes.FlinkConventions -import org.apache.flink.table.planner.plan.nodes.logical.{FlinkLogicalCalc, FlinkLogicalLegacyTableSourceScan} +import org.apache.flink.table.planner.plan.nodes.logical.{FlinkLogicalCalc, FlinkLogicalTableSourceScan} import org.apache.flink.table.planner.plan.optimize.program._ import org.apache.flink.table.planner.plan.rules.FlinkBatchRuleSets import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.NonDeterministicUdf @@ -54,7 +54,7 @@ class FlinkCalcMergeRuleTest extends TableTestBase { CoreRules.PROJECT_TO_CALC, FlinkCalcMergeRule.INSTANCE, FlinkLogicalCalc.CONVERTER, - FlinkLogicalLegacyTableSourceScan.CONVERTER + FlinkLogicalTableSourceScan.CONVERTER )) .setRequiredOutputTraits(Array(FlinkConventions.LOGICAL)) .build() diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PruneAggregateCallRuleTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PruneAggregateCallRuleTestBase.scala index 7891a03d9ec1c..868525bb5e9dc 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PruneAggregateCallRuleTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PruneAggregateCallRuleTestBase.scala @@ -17,13 +17,9 @@ */ package org.apache.flink.table.planner.plan.rules.logical -import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.api._ -import org.apache.flink.table.legacy.api.Types -import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.utils.{BatchTableTestUtil, TableTestBase} -import com.google.common.collect.ImmutableSet import org.junit.jupiter.api.{BeforeEach, Test} /** Base test class for [[PruneAggregateCallRule]]. */ @@ -32,12 +28,17 @@ abstract class PruneAggregateCallRuleTestBase extends TableTestBase { @BeforeEach def setup(): Unit = { - util.addTableSource( - "T1", - Array[TypeInformation[_]](Types.INT, Types.INT, Types.STRING, Types.INT), - Array("a1", "b1", "c1", "d1"), - FlinkStatistic.builder().uniqueKeys(ImmutableSet.of(ImmutableSet.of("a1"))).build() - ) + util.tableEnv.executeSql(s""" + |CREATE TABLE T1 ( + | a1 INT PRIMARY KEY NOT ENFORCED, + | b1 INT, + | c1 STRING, + | d1 BIGINT + |) WITH ( + | 'connector' = 'values', + | 'bounded' = 'true' + |) + |""".stripMargin) util.addTableSource[(Int, Int, String, Long)]("T2", 'a2, 'b2, 'c2, 'd2) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubqueryCorrelateVariablesValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubqueryCorrelateVariablesValidationTest.scala index 01064e579783b..7505d018901c8 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubqueryCorrelateVariablesValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubqueryCorrelateVariablesValidationTest.scala @@ -26,7 +26,8 @@ import java.sql.{Date, Timestamp} class SubqueryCorrelateVariablesValidationTest extends SubQueryTestBase { - util.addTableSource[(String, Short, Int, Long, Float, Double, BigDecimal, Timestamp, Date)]( + util.addTableSource[ + (String, Short, Int, Long, Float, Double, java.math.BigDecimal, Timestamp, Date)]( "t1", 't1a, 't1b, @@ -37,7 +38,8 @@ class SubqueryCorrelateVariablesValidationTest extends SubQueryTestBase { 't1g, 't1h, 't1i) - util.addTableSource[(String, Short, Int, Long, Float, Double, BigDecimal, Timestamp, Date)]( + util.addTableSource[ + (String, Short, Int, Long, Float, Double, java.math.BigDecimal, Timestamp, Date)]( "t2", 't2a, 't2b, @@ -48,7 +50,8 @@ class SubqueryCorrelateVariablesValidationTest extends SubQueryTestBase { 't2g, 't2h, 't2i) - util.addTableSource[(String, Short, Int, Long, Float, Double, BigDecimal, Timestamp, Date)]( + util.addTableSource[ + (String, Short, Int, Long, Float, Double, java.math.BigDecimal, Timestamp, Date)]( "t3", 't3a, 't3b, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/CalcTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/CalcTest.scala index 76c053d61931e..13b451aea2b6b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/CalcTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/CalcTest.scala @@ -17,13 +17,12 @@ */ package org.apache.flink.table.planner.plan.stream.sql -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.java.typeutils.TypeExtractor import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.utils.MyPojo import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.NonDeterministicUdf import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedTableFunctions.StringSplit import org.apache.flink.table.planner.utils.TableTestBase +import org.apache.flink.table.types.AbstractDataType import org.assertj.core.api.Assertions.assertThatExceptionOfType import org.junit.jupiter.api.{BeforeEach, Test} @@ -157,7 +156,7 @@ class CalcTest extends TableTestBase { def testPojoType(): Unit = { util.addTableSource( "MyTable4", - Array[TypeInformation[_]](TypeExtractor.createTypeInfo(classOf[MyPojo])), + Array[AbstractDataType[_]](DataTypes.RAW(classOf[MyPojo])), Array("a")) util.verifyExecPlan("SELECT a FROM MyTable4") } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.scala index a84f2d00a18a4..26efce18f27a4 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.scala @@ -17,11 +17,10 @@ */ package org.apache.flink.table.planner.plan.stream.sql -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.java.typeutils.GenericTypeInfo import org.apache.flink.table.api._ import org.apache.flink.table.planner.plan.utils.NonPojo import org.apache.flink.table.planner.utils.TableTestBase +import org.apache.flink.table.types.AbstractDataType import org.assertj.core.api.Assertions.assertThatExceptionOfType import org.junit.jupiter.api.{BeforeEach, Test} @@ -125,10 +124,11 @@ class SetOperatorsTest extends TableTestBase { val util = batchTestUtil() util.addTableSource( "A", - Array[TypeInformation[_]]( - new GenericTypeInfo(classOf[NonPojo]), - new GenericTypeInfo(classOf[NonPojo])), - Array("a", "b")) + Array[AbstractDataType[_]]( + DataTypes.STRUCTURED(classOf[NonPojo]), + DataTypes.STRUCTURED(classOf[NonPojo])), + Array("a", "b") + ) util.verifyExecPlan("SELECT a FROM A UNION ALL SELECT b FROM A") } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala index 674355e121946..eb533cbe1c0ef 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala @@ -32,12 +32,6 @@ class TableScanTest extends TableTestBase { private val util = streamTestUtil() - @Test - def testLegacyTableSourceScan(): Unit = { - util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) - util.verifyExecPlan("SELECT * FROM MyTable") - } - @Test def testDataStreamScan(): Unit = { util.addDataStream[(Int, Long, String)]("DataStreamTable", 'a, 'b, 'c) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala index 8e38caab8da50..0d9a6d62ccdc2 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala @@ -17,12 +17,10 @@ */ package org.apache.flink.table.planner.plan.stream.sql.agg -import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.api._ import org.apache.flink.table.api.config.ExecutionConfigOptions -import org.apache.flink.table.legacy.api.Types import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase} -import org.apache.flink.table.runtime.typeutils.DecimalDataTypeInfo +import org.apache.flink.table.types.AbstractDataType import org.assertj.core.api.Assertions.{assertThatExceptionOfType, assertThatThrownBy} import org.junit.jupiter.api.Test @@ -44,20 +42,20 @@ class AggregateTest extends TableTestBase { util.addTableSource[(Long, Int, String)]("T2", 'a, 'b, 'c) util.addTableSource( "MyTable1", - Array[TypeInformation[_]]( - Types.BYTE, - Types.SHORT, - Types.INT, - Types.LONG, - Types.FLOAT, - Types.DOUBLE, - Types.BOOLEAN, - Types.STRING, - Types.LOCAL_DATE, - Types.LOCAL_TIME, - Types.LOCAL_DATE_TIME, - DecimalDataTypeInfo.of(30, 20), - DecimalDataTypeInfo.of(10, 5) + Array[AbstractDataType[_]]( + DataTypes.TINYINT, + DataTypes.SMALLINT, + DataTypes.INT, + DataTypes.BIGINT, + DataTypes.FLOAT, + DataTypes.DOUBLE, + DataTypes.BOOLEAN, + DataTypes.STRING, + DataTypes.DATE, + DataTypes.TIME(0), + DataTypes.TIMESTAMP(3), + DataTypes.DECIMAL(30, 20), + DataTypes.DECIMAL(10, 5) ), Array( "byte", diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/OverWindowValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/OverWindowValidationTest.scala index a3df1134dcd8b..a2eef8e0f3d91 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/OverWindowValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/OverWindowValidationTest.scala @@ -130,7 +130,7 @@ class OverWindowValidationTest extends TableTestBase { @Test def testPartitionByWithNotKeyType(): Unit = { val table2 = - streamUtil.addTableSource[(Int, String, Either[Long, String])]("MyTable2", 'a, 'b, 'c) + streamUtil.addTableSource[(Int, String, (Long, String))]("MyTable2", 'a, 'b, 'c) assertThatExceptionOfType(classOf[ValidationException]) .isThrownBy( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala index 973ad7640e439..1850efa34617e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala @@ -18,10 +18,12 @@ package org.apache.flink.table.planner.utils import org.apache.flink.FlinkVersion -import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation} +import org.apache.flink.api.common.serialization.SerializerConfigImpl +import org.apache.flink.api.common.typeinfo.{AtomicType, BasicArrayTypeInfo, BasicTypeInfo, TypeInformation} +import org.apache.flink.api.common.typeutils.TypeSerializer import org.apache.flink.api.dag.Transformation import org.apache.flink.api.java.typeutils.{PojoTypeInfo, RowTypeInfo, TupleTypeInfo} -import org.apache.flink.configuration.BatchExecutionOptions +import org.apache.flink.configuration.{BatchExecutionOptions, ConfigOption, ConfigOptions} import org.apache.flink.legacy.table.factories.StreamTableSourceFactory import org.apache.flink.legacy.table.sinks.{AppendStreamTableSink, RetractStreamTableSink, UpsertStreamTableSink} import org.apache.flink.legacy.table.sources.StreamTableSource @@ -30,6 +32,7 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.environment import org.apache.flink.streaming.api.environment.{LocalStreamEnvironment, StreamExecutionEnvironment} +import org.apache.flink.streaming.api.legacy.io.CollectionInputFormat import org.apache.flink.table.api._ import org.apache.flink.table.api.bridge.java.{StreamTableEnvironment => JavaStreamTableEnv} import org.apache.flink.table.api.bridge.scala.{StreamTableEnvironment => ScalaStreamTableEnv} @@ -37,12 +40,14 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.api.internal.{StatementSetImpl, TableEnvironmentImpl, TableEnvironmentInternal, TableImpl} import org.apache.flink.table.api.typeutils.CaseClassTypeInfo import org.apache.flink.table.catalog._ -import org.apache.flink.table.data.RowData +import org.apache.flink.table.connector.ChangelogMode +import org.apache.flink.table.connector.source.{DynamicTableSource, InputFormatProvider, ScanTableSource} +import org.apache.flink.table.data.{DecimalDataUtils, RowData} import org.apache.flink.table.delegation.{Executor, ExecutorFactory} import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE import org.apache.flink.table.descriptors.DescriptorProperties import org.apache.flink.table.expressions.Expression -import org.apache.flink.table.factories.{FactoryUtil, PlannerFactoryUtil} +import org.apache.flink.table.factories._ import org.apache.flink.table.functions._ import org.apache.flink.table.legacy.api.TableSchema import org.apache.flink.table.legacy.descriptors.Schema.SCHEMA @@ -52,6 +57,7 @@ import org.apache.flink.table.module.ModuleManager import org.apache.flink.table.operations.{ModifyOperation, QueryOperation} import org.apache.flink.table.planner.calcite.CalciteConfig import org.apache.flink.table.planner.delegation.PlannerBase +import org.apache.flink.table.planner.factories.TestValuesTableFactory import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable import org.apache.flink.table.planner.operations.{InternalDataStreamQueryOperation, PlannerQueryOperation, RichTableSourceQueryOperation} import org.apache.flink.table.planner.plan.nodes.calcite.LogicalWatermarkAssigner @@ -65,11 +71,14 @@ import org.apache.flink.table.planner.runtime.utils.{TestingAppendTableSink, Tes import org.apache.flink.table.planner.sinks.CollectRowTableSink import org.apache.flink.table.planner.utils.PlanKind.PlanKind import org.apache.flink.table.planner.utils.TableTestUtil.{replaceNodeIdInOperator, replaceStageId, replaceStreamNodeId} +import org.apache.flink.table.planner.utils.TestSimpleDynamicTableSourceFactory.{BOUNDED, IDENTIFIER} import org.apache.flink.table.resource.ResourceManager import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromLogicalTypeToTypeInfo -import org.apache.flink.table.types.logical.LogicalType -import org.apache.flink.table.types.utils.TypeConversions -import org.apache.flink.table.typeutils.FieldInfoUtils +import org.apache.flink.table.types._ +import org.apache.flink.table.types.logical.{LegacyTypeInformationType, LogicalType, LogicalTypeRoot} +import org.apache.flink.table.types.utils.{LegacyTypeInfoDataTypeConverter, TypeConversions} +import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType +import org.apache.flink.table.typeutils.{FieldInfoUtils, TimeIndicatorTypeInfo} import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension import org.apache.flink.types.Row import org.apache.flink.util.{FlinkUserCodeClassLoaders, MutableURLClassLoader} @@ -96,6 +105,7 @@ import java.time.Duration import java.util.Collections import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer /** Test base for testing Table API / SQL plans. */ abstract class TableTestBase { @@ -228,8 +238,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) } /** - * Create a [[TestTableSource]] with the given schema, and registers this TableSource under given - * name into the TableEnvironment's catalog. + * Create a [[TestSimpleDynamicTableSource]] with the given schema, and registers this TableSource + * under given name into the TableEnvironment's catalog. * * @param name * table name @@ -253,18 +263,121 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) } val types = fieldTypes.map(TypeConversions.fromLegacyInfoToDataType) val names = FieldInfoUtils.getFieldNames(typeInfo) - TableSchema.builder().fields(names, types).build() + TableSchema.builder().fields(names, types).build().toSchema } else { - TableSchema.fromResolvedSchema( - FieldInfoUtils.getFieldsInfo(typeInfo, fields.toArray).toResolvedSchema) + val typeInfoSchema = FieldInfoUtils.getFieldsInfo(typeInfo, fields.toArray) + + val schemaBuilder = Schema.newBuilder() + val fieldNames = typeInfoSchema.getFieldNames + val fieldTypes = typeInfoSchema.getFieldTypes + val fieldIndices = typeInfoSchema.getIndices + + (0 until fieldIndices.length).foreach( + idx => { + val fieldIndex = fieldIndices(idx) + if ( + fieldIndex == TimeIndicatorTypeInfo.PROCTIME_STREAM_MARKER || + fieldIndex == TimeIndicatorTypeInfo.PROCTIME_BATCH_MARKER + ) { + schemaBuilder.columnByExpression(fieldNames(idx), "PROCTIME()") + } else if ( + fieldIndex == TimeIndicatorTypeInfo.ROWTIME_STREAM_MARKER || + fieldIndex == TimeIndicatorTypeInfo.ROWTIME_BATCH_MARKER + ) { + schemaBuilder.column(fieldNames(idx), fieldTypes(idx)) + schemaBuilder.watermark(fieldNames(idx), s"${fieldNames(idx)}") + } else { + // Fallback to original behavior to keep compatibility. + // See more at FieldInfoUtils#toResolvedSchema. + schemaBuilder.column(fieldNames(idx), resolveLegacyTypeInfo(fieldTypes(idx))) + } + }) + schemaBuilder.build() } - addTableSource(name, new TestTableSource(isBounded, tableSchema)) + addTableSource(name, tableSchema) } /** - * Create a [[TestTableSource]] with the given schema, table stats and unique keys, and registers - * this TableSource under given name into the TableEnvironment's catalog. + * This is a temporary solution to maintain compatibility with the old type system in tests. See + * more at [[LegacyTypeInfoDataTypeConverter]]. + * + * Currently, we support to resolve the following common legacy types in tests: + * + * - CaseClassTypeInfo + * - BasicArrayTypeInfo + * - java.math.BigDecimal + */ + private def resolveLegacyTypeInfo(dataType: DataType): DataType = { + val visitor = new DataTypeVisitor[DataType] { + override def visit(atomicDataType: AtomicDataType): DataType = { + if (!atomicDataType.getLogicalType.isInstanceOf[LegacyTypeInformationType[_]]) { + return atomicDataType + } + + val legacyType = atomicDataType.getLogicalType.asInstanceOf[LegacyTypeInformationType[_]] + + // resolve CaseClassTypeInfo + if (legacyType.getTypeInformation.isInstanceOf[CaseClassTypeInfo[_]]) { + val innerType = atomicDataType.getLogicalType + .asInstanceOf[LegacyTypeInformationType[_]] + .getTypeInformation + .asInstanceOf[CaseClassTypeInfo[_]] + + val innerFieldNames = innerType.getFieldNames + val innerFieldTypes = innerType.getFieldTypes + + val rowFields = ArrayBuffer[DataTypes.Field]() + (0 until innerFieldNames.length).foreach( + i => { + rowFields += DataTypes.FIELD( + innerFieldNames(i), + resolveLegacyTypeInfo(fromLegacyInfoToDataType(innerFieldTypes(i)))) + }) + DataTypes.ROW(rowFields: _*) + } + // resolve BasicArrayTypeInfo + else if (legacyType.getTypeInformation.isInstanceOf[BasicArrayTypeInfo[_, _]]) { + val arrayType = legacyType.getTypeInformation.asInstanceOf[BasicArrayTypeInfo[_, _]] + DataTypes.ARRAY( + resolveLegacyTypeInfo(fromLegacyInfoToDataType(arrayType.getComponentInfo))) + } + // resolve java.math.BigDecimal + else if ( + legacyType.getTypeInformation + .isInstanceOf[BasicTypeInfo[_]] && legacyType.getTypeRoot == LogicalTypeRoot.DECIMAL + ) { + new AtomicDataType(DecimalDataUtils.DECIMAL_SYSTEM_DEFAULT) + } else { + throw new UnsupportedOperationException(s"Unsupported legacy type info: $legacyType") + } + } + + override def visit(collectionDataType: CollectionDataType): DataType = { + val elementType = collectionDataType.getElementDataType.accept(this) + new CollectionDataType( + DataTypes.ARRAY(elementType).getLogicalType, + collectionDataType.getElementDataType.accept(this)) + } + + override def visit(fieldsDataType: FieldsDataType): DataType = { + fieldsDataType + } + + override def visit(keyValueDataType: KeyValueDataType): DataType = { + val keyType = keyValueDataType.getKeyDataType.accept(this) + val valueType = keyValueDataType.getValueDataType.accept(this) + new KeyValueDataType(DataTypes.MAP(keyType, valueType).getLogicalType, keyType, valueType) + } + } + + dataType.accept(visitor) + + } + + /** + * Create a [[TestSimpleDynamicTableSource]] with the given schema, and registers this TableSource + * under given name into the TableEnvironment's catalog. * * @param name * table name @@ -277,11 +390,10 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) */ def addTableSource( name: String, - types: Array[TypeInformation[_]], + types: Array[AbstractDataType[_]], fields: Array[String]): Table = { - val schema = new TableSchema(fields, types) - val tableSource = new TestTableSource(isBounded, schema) - addTableSource(name, tableSource) + val schema = Schema.newBuilder().fromFields(fields, types).build() + addTableSource(name, schema) } /** @@ -289,15 +401,32 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) * * @param name * table name - * @param tableSource - * table source + * @param schema + * table schema * @return * returns the registered [[Table]]. */ - def addTableSource(name: String, tableSource: TableSource[_]): Table = { - getTableEnv + def addTableSource( + name: String, + schema: Schema + ): Table = { + val options = new util.HashMap[String, String]() + options.put("connector", IDENTIFIER) + options.put(BOUNDED.key(), isBounded.toString) + + val table = CatalogTable.newBuilder().schema(schema).options(options).build() + val catalogManager = getTableEnv .asInstanceOf[TableEnvironmentInternal] - .registerTableSourceInternal(name, tableSource) + .getCatalogManager + + catalogManager.createTable( + table, + ObjectIdentifier.of( + catalogManager.getCurrentCatalog, + catalogManager.getCurrentDatabase, + name), + false) + getTableEnv.from(name) } @@ -1493,6 +1622,63 @@ class TestTableSourceFactory extends StreamTableSourceFactory[Row] { } } +/** + * Different with table in [[TestValuesTableFactory]], this table source does not support all + * features like agg pushdown, filter pushdown, etc. + */ +class TestSimpleDynamicTableSource(bounded: Boolean, producedDataType: DataType) + extends ScanTableSource { + + override def getChangelogMode: ChangelogMode = ChangelogMode.insertOnly() + + override def getScanRuntimeProvider( + runtimeProviderContext: ScanTableSource.ScanContext): ScanTableSource.ScanRuntimeProvider = { + + val dataType: TypeInformation[RowData] = + runtimeProviderContext.createTypeInformation(producedDataType) + val serializer: TypeSerializer[RowData] = dataType.createSerializer(new SerializerConfigImpl) + // use InputFormatProvider here to ensure the source is not chainable + InputFormatProvider.of(new CollectionInputFormat[RowData](Collections.emptyList(), serializer)) + } + + override def copy(): DynamicTableSource = { + new TestSimpleDynamicTableSource(bounded, producedDataType) + } + + override def asSummaryString(): String = { + "TestSimpleDynamicTableSource" + } +} + +class TestSimpleDynamicTableSourceFactory extends DynamicTableSourceFactory { + + override def createDynamicTableSource( + context: DynamicTableFactory.Context): DynamicTableSource = { + val helper = FactoryUtil.createTableFactoryHelper(this, context); + helper.validate() + val isBounded = helper.getOptions.get(BOUNDED) + val producedDataType = context.getPhysicalRowDataType + new TestSimpleDynamicTableSource(isBounded, producedDataType) + } + + override def factoryIdentifier(): String = IDENTIFIER + + override def requiredOptions(): util.Set[ConfigOption[_]] = { + new util.HashSet(util.Arrays.asList(BOUNDED)) + } + + override def optionalOptions(): util.Set[ConfigOption[_]] = { + Collections.emptySet() + } +} + +object TestSimpleDynamicTableSourceFactory { + val IDENTIFIER = "test-dynamic-table-source" + + val BOUNDED: ConfigOption[java.lang.Boolean] = + ConfigOptions.key("bounded").booleanType().noDefaultValue() +} + class TestingTableEnvironment private ( catalogManager: CatalogManager, moduleManager: ModuleManager, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/testTableSourceSinks.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/testTableSourceSinks.scala index 432fc3f4f67cd..008d81f1b8466 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/testTableSourceSinks.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/testTableSourceSinks.scala @@ -240,17 +240,6 @@ class TestStreamTableSource(tableSchema: TableSchema, values: Seq[Row]) override def getTableSchema: TableSchema = tableSchema } -object TestStreamTableSource { - def createTemporaryTable( - tEnv: TableEnvironment, - schema: TableSchema, - tableName: String, - data: Seq[Row] = null): Unit = { - val source = new TestStreamTableSource(schema, data) - tEnv.asInstanceOf[TableEnvironmentInternal].registerTableSourceInternal(tableName, source) - } -} - class TestStreamTableSourceFactory extends StreamTableSourceFactory[Row] { override def createStreamTableSource(properties: JMap[String, String]): StreamTableSource[Row] = { val descriptorProperties = new DescriptorProperties