diff --git a/datafusion/core/tests/sqllogictests/test_files/subquery.slt b/datafusion/core/tests/sqllogictests/test_files/subquery.slt index a0720d1c3aeb..3ab8d83c6958 100644 --- a/datafusion/core/tests/sqllogictests/test_files/subquery.slt +++ b/datafusion/core/tests/sqllogictests/test_files/subquery.slt @@ -973,3 +973,14 @@ false false true true + + +# issue: https://github.com/apache/arrow-datafusion/issues/7027 +query TTTT rowsort +SELECT * FROM + (VALUES ('catan-prod1-daily', 'success')) as jobs(cron_job_name, status) + JOIN + (VALUES ('catan-prod1-daily', 'high')) as severity(cron_job_name, level) + ON (severity.cron_job_name = jobs.cron_job_name); +---- +catan-prod1-daily success catan-prod1-daily high diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q13.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q13.slt.part index 732ae27f0108..b65f56c891a1 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q13.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q13.slt.part @@ -41,19 +41,18 @@ limit 10; ---- logical_plan Limit: skip=0, fetch=10 ---Sort: custdist DESC NULLS FIRST, c_count DESC NULLS FIRST, fetch=10 -----Projection: c_count, COUNT(UInt8(1)) AS custdist -------Aggregate: groupBy=[[c_count]], aggr=[[COUNT(UInt8(1))]] ---------Projection: c_orders.COUNT(orders.o_orderkey) AS c_count -----------SubqueryAlias: c_orders -------------Projection: COUNT(orders.o_orderkey) ---------------Aggregate: groupBy=[[customer.c_custkey]], aggr=[[COUNT(orders.o_orderkey)]] -----------------Projection: customer.c_custkey, orders.o_orderkey -------------------Left Join: customer.c_custkey = orders.o_custkey ---------------------TableScan: customer projection=[c_custkey] ---------------------Projection: orders.o_orderkey, orders.o_custkey -----------------------Filter: orders.o_comment NOT LIKE Utf8("%special%requests%") -------------------------TableScan: orders projection=[o_orderkey, o_custkey, o_comment], partial_filters=[orders.o_comment NOT LIKE Utf8("%special%requests%")] +--Sort: custdist DESC NULLS FIRST, c_orders.c_count DESC NULLS FIRST, fetch=10 +----Projection: c_orders.c_count, COUNT(UInt8(1)) AS custdist +------Aggregate: groupBy=[[c_orders.c_count]], aggr=[[COUNT(UInt8(1))]] +--------SubqueryAlias: c_orders +----------Projection: COUNT(orders.o_orderkey) AS c_count +------------Aggregate: groupBy=[[customer.c_custkey]], aggr=[[COUNT(orders.o_orderkey)]] +--------------Projection: customer.c_custkey, orders.o_orderkey +----------------Left Join: customer.c_custkey = orders.o_custkey +------------------TableScan: customer projection=[c_custkey] +------------------Projection: orders.o_orderkey, orders.o_custkey +--------------------Filter: orders.o_comment NOT LIKE Utf8("%special%requests%") +----------------------TableScan: orders projection=[o_orderkey, o_custkey, o_comment], partial_filters=[orders.o_comment NOT LIKE Utf8("%special%requests%")] physical_plan GlobalLimitExec: skip=0, fetch=10 --SortPreservingMergeExec: [custdist@1 DESC,c_count@0 DESC], fetch=10 @@ -63,22 +62,21 @@ GlobalLimitExec: skip=0, fetch=10 ----------CoalesceBatchesExec: target_batch_size=8192 ------------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4 --------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[COUNT(UInt8(1))] -----------------ProjectionExec: expr=[COUNT(orders.o_orderkey)@0 as c_count] -------------------ProjectionExec: expr=[COUNT(orders.o_orderkey)@1 as COUNT(orders.o_orderkey)] ---------------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)] -----------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, o_orderkey@1 as o_orderkey] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 ---------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 ---------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests% ---------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false +----------------ProjectionExec: expr=[COUNT(orders.o_orderkey)@1 as c_count] +------------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)] +--------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, o_orderkey@1 as o_orderkey] +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)] +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] +--------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests% +------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q15.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q15.slt.part index 613f44f2970d..4dea89fc72f1 100644 --- a/datafusion/core/tests/sqllogictests/test_files/tpch/q15.slt.part +++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q15.slt.part @@ -51,32 +51,31 @@ order by ---- logical_plan Sort: supplier.s_suppkey ASC NULLS LAST ---Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, total_revenue -----Inner Join: total_revenue = __scalar_sq_1.MAX(total_revenue) -------Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, total_revenue ---------Inner Join: supplier.s_suppkey = supplier_no +--Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue0.total_revenue +----Inner Join: revenue0.total_revenue = __scalar_sq_1.MAX(revenue0.total_revenue) +------Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue0.total_revenue +--------Inner Join: supplier.s_suppkey = revenue0.supplier_no ----------TableScan: supplier projection=[s_suppkey, s_name, s_address, s_phone] -----------Projection: revenue0.l_suppkey AS supplier_no, revenue0.SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS total_revenue -------------SubqueryAlias: revenue0 +----------SubqueryAlias: revenue0 +------------Projection: lineitem.l_suppkey AS supplier_no, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS total_revenue --------------Aggregate: groupBy=[[lineitem.l_suppkey]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] ----------------Projection: lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount ------------------Filter: lineitem.l_shipdate >= Date32("9496") AND lineitem.l_shipdate < Date32("9587") --------------------TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9496"), lineitem.l_shipdate < Date32("9587")] ------SubqueryAlias: __scalar_sq_1 ---------Aggregate: groupBy=[[]], aggr=[[MAX(total_revenue)]] -----------Projection: revenue0.SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS total_revenue -------------SubqueryAlias: revenue0 ---------------Projection: SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) -----------------Aggregate: groupBy=[[lineitem.l_suppkey]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] -------------------Projection: lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount ---------------------Filter: lineitem.l_shipdate >= Date32("9496") AND lineitem.l_shipdate < Date32("9587") -----------------------TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9496"), lineitem.l_shipdate < Date32("9587")] +--------Aggregate: groupBy=[[]], aggr=[[MAX(revenue0.total_revenue)]] +----------SubqueryAlias: revenue0 +------------Projection: SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS total_revenue +--------------Aggregate: groupBy=[[lineitem.l_suppkey]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] +----------------Projection: lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount +------------------Filter: lineitem.l_shipdate >= Date32("9496") AND lineitem.l_shipdate < Date32("9587") +--------------------TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9496"), lineitem.l_shipdate < Date32("9587")] physical_plan SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] --SortExec: expr=[s_suppkey@0 ASC NULLS LAST] ----ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address, s_phone@3 as s_phone, total_revenue@4 as total_revenue] ------CoalesceBatchesExec: target_batch_size=8192 ---------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(total_revenue@4, MAX(total_revenue)@0)] +--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(total_revenue@4, MAX(revenue0.total_revenue)@0)] ----------CoalesceBatchesExec: target_batch_size=8192 ------------RepartitionExec: partitioning=Hash([total_revenue@4], 4), input_partitions=4 --------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address, s_phone@3 as s_phone, total_revenue@5 as total_revenue] @@ -96,20 +95,19 @@ SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] ----------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 ------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false ----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([MAX(total_revenue)@0], 4), input_partitions=1 ---------------AggregateExec: mode=Final, gby=[], aggr=[MAX(total_revenue)] +------------RepartitionExec: partitioning=Hash([MAX(revenue0.total_revenue)@0], 4), input_partitions=1 +--------------AggregateExec: mode=Final, gby=[], aggr=[MAX(revenue0.total_revenue)] ----------------CoalescePartitionsExec -------------------AggregateExec: mode=Partial, gby=[], aggr=[MAX(total_revenue)] ---------------------ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as total_revenue] -----------------------ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -------------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -------------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ---------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 ---------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false +------------------AggregateExec: mode=Partial, gby=[], aggr=[MAX(revenue0.total_revenue)] +--------------------ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +----------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 +----------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +--------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 +------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false query ITTTR diff --git a/datafusion/sql/src/planner.rs b/datafusion/sql/src/planner.rs index 26ff5466f408..efa3077c3b59 100644 --- a/datafusion/sql/src/planner.rs +++ b/datafusion/sql/src/planner.rs @@ -34,7 +34,7 @@ use datafusion_common::{OwnedTableReference, TableReference}; use datafusion_expr::logical_plan::{LogicalPlan, LogicalPlanBuilder}; use datafusion_expr::utils::find_column_exprs; use datafusion_expr::TableSource; -use datafusion_expr::{col, AggregateUDF, Expr, ScalarUDF, SubqueryAlias}; +use datafusion_expr::{col, AggregateUDF, Expr, ScalarUDF}; use crate::utils::make_decimal_type; @@ -222,18 +222,17 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Ok(Schema::new(fields)) } - /// Apply the given TableAlias to the top-level projection. + /// Apply the given TableAlias to the input plan pub(crate) fn apply_table_alias( &self, plan: LogicalPlan, alias: TableAlias, ) -> Result { - let apply_name_plan = LogicalPlan::SubqueryAlias(SubqueryAlias::try_new( - plan, - self.normalizer.normalize(alias.name), - )?); + let plan = self.apply_expr_alias(plan, alias.columns)?; - self.apply_expr_alias(apply_name_plan, alias.columns) + LogicalPlanBuilder::from(plan) + .alias(self.normalizer.normalize(alias.name))? + .build() } pub(crate) fn apply_expr_alias( diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index 652011fbac31..6b498084a498 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -617,9 +617,9 @@ fn select_nested_with_filters() { fn table_with_column_alias() { let sql = "SELECT a, b, c FROM lineitem l (a, b, c)"; - let expected = "Projection: a, b, c\ - \n Projection: l.l_item_id AS a, l.l_description AS b, l.price AS c\ - \n SubqueryAlias: l\ + let expected = "Projection: l.a, l.b, l.c\ + \n SubqueryAlias: l\ + \n Projection: lineitem.l_item_id AS a, lineitem.l_description AS b, lineitem.price AS c\ \n TableScan: lineitem"; quick_test(sql, expected); @@ -1186,9 +1186,9 @@ fn select_simple_aggregate_repeated_aggregate_with_unique_aliases() { fn select_from_typed_string_values() { quick_test( "SELECT col1, col2 FROM (VALUES (TIMESTAMP '2021-06-10 17:01:00Z', DATE '2004-04-09')) as t (col1, col2)", - "Projection: col1, col2\ - \n Projection: t.column1 AS col1, t.column2 AS col2\ - \n SubqueryAlias: t\ + "Projection: t.col1, t.col2\ + \n SubqueryAlias: t\ + \n Projection: column1 AS col1, column2 AS col2\ \n Values: (CAST(Utf8(\"2021-06-10 17:01:00Z\") AS Timestamp(Nanosecond, None)), CAST(Utf8(\"2004-04-09\") AS Date32))", ); } @@ -2969,9 +2969,9 @@ fn cte_with_column_names() { ) \ SELECT * FROM numbers;"; - let expected = "Projection: a, b, c\ - \n Projection: numbers.Int64(1) AS a, numbers.Int64(2) AS b, numbers.Int64(3) AS c\ - \n SubqueryAlias: numbers\ + let expected = "Projection: numbers.a, numbers.b, numbers.c\ + \n SubqueryAlias: numbers\ + \n Projection: Int64(1) AS a, Int64(2) AS b, Int64(3) AS c\ \n Projection: Int64(1), Int64(2), Int64(3)\ \n EmptyRelation"; @@ -2987,9 +2987,9 @@ fn cte_with_column_aliases_precedence() { ) \ SELECT * FROM numbers;"; - let expected = "Projection: a, b, c\ - \n Projection: numbers.x AS a, numbers.y AS b, numbers.z AS c\ - \n SubqueryAlias: numbers\ + let expected = "Projection: numbers.a, numbers.b, numbers.c\ + \n SubqueryAlias: numbers\ + \n Projection: x AS a, y AS b, z AS c\ \n Projection: Int64(1) AS x, Int64(2) AS y, Int64(3) AS z\ \n EmptyRelation"; quick_test(sql, expected) @@ -4015,9 +4015,9 @@ fn test_prepare_statement_to_plan_value_list() { let sql = "PREPARE my_plan(STRING, STRING) AS SELECT * FROM (VALUES(1, $1), (2, $2)) AS t (num, letter);"; let expected_plan = "Prepare: \"my_plan\" [Utf8, Utf8] \ - \n Projection: num, letter\ - \n Projection: t.column1 AS num, t.column2 AS letter\ - \n SubqueryAlias: t\ + \n Projection: t.num, t.letter\ + \n SubqueryAlias: t\ + \n Projection: column1 AS num, column2 AS letter\ \n Values: (Int64(1), $1), (Int64(2), $2)"; let expected_dt = "[Utf8, Utf8]"; @@ -4030,9 +4030,9 @@ fn test_prepare_statement_to_plan_value_list() { ScalarValue::Utf8(Some("a".to_string())), ScalarValue::Utf8(Some("b".to_string())), ]; - let expected_plan = "Projection: num, letter\ - \n Projection: t.column1 AS num, t.column2 AS letter\ - \n SubqueryAlias: t\ + let expected_plan = "Projection: t.num, t.letter\ + \n SubqueryAlias: t\ + \n Projection: column1 AS num, column2 AS letter\ \n Values: (Int64(1), Utf8(\"a\")), (Int64(2), Utf8(\"b\"))"; prepare_stmt_replace_params_quick_test(plan, param_values, expected_plan); @@ -4063,9 +4063,9 @@ fn test_table_alias() { (select age from person) t2 \ ) as f (c1, c2)"; - let expected = "Projection: c1, c2\ - \n Projection: f.id AS c1, f.age AS c2\ - \n SubqueryAlias: f\ + let expected = "Projection: f.c1, f.c2\ + \n SubqueryAlias: f\ + \n Projection: t1.id AS c1, t2.age AS c2\ \n CrossJoin:\ \n SubqueryAlias: t1\ \n Projection: person.id\