diff --git a/benchmarks/expected-plans/q18.txt b/benchmarks/expected-plans/q18.txt index 639598725ce04..01f60ba55f315 100644 --- a/benchmarks/expected-plans/q18.txt +++ b/benchmarks/expected-plans/q18.txt @@ -1,14 +1,13 @@ Sort: orders.o_totalprice DESC NULLS FIRST, orders.o_orderdate ASC NULLS LAST - Projection: customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, SUM(lineitem.l_quantity) - Aggregate: groupBy=[[customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice]], aggr=[[SUM(lineitem.l_quantity)]] - LeftSemi Join: orders.o_orderkey = __correlated_sq_1.l_orderkey - Inner Join: orders.o_orderkey = lineitem.l_orderkey - Inner Join: customer.c_custkey = orders.o_custkey - TableScan: customer projection=[c_custkey, c_name] - TableScan: orders projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate] - TableScan: lineitem projection=[l_orderkey, l_quantity] - SubqueryAlias: __correlated_sq_1 - Projection: lineitem.l_orderkey AS l_orderkey - Filter: SUM(lineitem.l_quantity) > Decimal128(Some(30000),25,2) - Aggregate: groupBy=[[lineitem.l_orderkey]], aggr=[[SUM(lineitem.l_quantity)]] - TableScan: lineitem projection=[l_orderkey, l_quantity] \ No newline at end of file + Aggregate: groupBy=[[customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice]], aggr=[[SUM(lineitem.l_quantity)]] + LeftSemi Join: orders.o_orderkey = __correlated_sq_1.l_orderkey + Inner Join: orders.o_orderkey = lineitem.l_orderkey + Inner Join: customer.c_custkey = orders.o_custkey + TableScan: customer projection=[c_custkey, c_name] + TableScan: orders projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate] + TableScan: lineitem projection=[l_orderkey, l_quantity] + SubqueryAlias: __correlated_sq_1 + Projection: lineitem.l_orderkey AS l_orderkey + Filter: SUM(lineitem.l_quantity) > Decimal128(Some(30000),25,2) + Aggregate: groupBy=[[lineitem.l_orderkey]], aggr=[[SUM(lineitem.l_quantity)]] + TableScan: lineitem projection=[l_orderkey, l_quantity] \ No newline at end of file diff --git a/benchmarks/expected-plans/q21.txt b/benchmarks/expected-plans/q21.txt index a91632df4e479..c1d7417d83183 100644 --- a/benchmarks/expected-plans/q21.txt +++ b/benchmarks/expected-plans/q21.txt @@ -14,9 +14,8 @@ Sort: numwait DESC NULLS FIRST, supplier.s_name ASC NULLS LAST TableScan: orders projection=[o_orderkey, o_orderstatus] Filter: nation.n_name = Utf8("SAUDI ARABIA") TableScan: nation projection=[n_nationkey, n_name] - Projection: l2.l_orderkey, l2.l_suppkey - SubqueryAlias: l2 - TableScan: lineitem projection=[l_orderkey, l_suppkey] + SubqueryAlias: l2 + TableScan: lineitem projection=[l_orderkey, l_suppkey] Projection: l3.l_orderkey, l3.l_suppkey SubqueryAlias: l3 Filter: lineitem.l_receiptdate > lineitem.l_commitdate diff --git a/benchmarks/expected-plans/q22.txt b/benchmarks/expected-plans/q22.txt index 11b438085a0bb..0fd7a590ac194 100644 --- a/benchmarks/expected-plans/q22.txt +++ b/benchmarks/expected-plans/q22.txt @@ -8,8 +8,7 @@ Sort: custsale.cntrycode ASC NULLS LAST LeftAnti Join: customer.c_custkey = orders.o_custkey Filter: substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")]) TableScan: customer projection=[c_custkey, c_phone, c_acctbal] - Projection: orders.o_custkey - TableScan: orders projection=[o_custkey] + TableScan: orders projection=[o_custkey] SubqueryAlias: __scalar_sq_1 Projection: AVG(customer.c_acctbal) AS __value Aggregate: groupBy=[[]], aggr=[[AVG(customer.c_acctbal)]] diff --git a/datafusion/core/src/datasource/view.rs b/datafusion/core/src/datasource/view.rs index 524ad9f5c2ad9..dea7ecadf7f4e 100644 --- a/datafusion/core/src/datasource/view.rs +++ b/datafusion/core/src/datasource/view.rs @@ -500,8 +500,7 @@ mod tests { let expected = "\ Explain\ \n CreateView: Bare { table: \"xyz\" }\ - \n Projection: abc.column1, abc.column2, abc.column3\ - \n TableScan: abc projection=[column1, column2, column3]"; + \n TableScan: abc projection=[column1, column2, column3]"; assert_eq!(expected, actual); let dataframe = session_ctx @@ -512,9 +511,8 @@ mod tests { let expected = "\ Explain\ \n CreateView: Bare { table: \"xyz\" }\ - \n Projection: abc.column1, abc.column2, abc.column3\ - \n Filter: abc.column2 = Int64(5)\ - \n TableScan: abc projection=[column1, column2, column3]"; + \n Filter: abc.column2 = Int64(5)\ + \n TableScan: abc projection=[column1, column2, column3]"; assert_eq!(expected, actual); let dataframe = session_ctx @@ -525,9 +523,8 @@ mod tests { let expected = "\ Explain\ \n CreateView: Bare { table: \"xyz\" }\ - \n Projection: abc.column1, abc.column2\ - \n Filter: abc.column2 = Int64(5)\ - \n TableScan: abc projection=[column1, column2]"; + \n Filter: abc.column2 = Int64(5)\ + \n TableScan: abc projection=[column1, column2]"; assert_eq!(expected, actual); Ok(()) diff --git a/datafusion/core/src/physical_plan/coalesce_batches.rs b/datafusion/core/src/physical_plan/coalesce_batches.rs index 2e7211fc3ae47..ec7dd7b4d63a2 100644 --- a/datafusion/core/src/physical_plan/coalesce_batches.rs +++ b/datafusion/core/src/physical_plan/coalesce_batches.rs @@ -295,7 +295,6 @@ mod tests { use crate::config::ConfigOptions; use crate::datasource::MemTable; use crate::physical_plan::filter::FilterExec; - use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::{memory::MemoryExec, repartition::RepartitionExec}; use crate::prelude::SessionContext; use crate::test::create_vec_batches; @@ -308,12 +307,7 @@ mod tests { let ctx = SessionContext::with_config(config.into()); let plan = create_physical_plan(ctx).await?; - let projection = plan.as_any().downcast_ref::().unwrap(); - let coalesce = projection - .input() - .as_any() - .downcast_ref::() - .unwrap(); + let coalesce = plan.as_any().downcast_ref::().unwrap(); assert_eq!(1234, coalesce.target_batch_size); Ok(()) } @@ -325,13 +319,7 @@ mod tests { let ctx = SessionContext::with_config(config.into()); let plan = create_physical_plan(ctx).await?; - let projection = plan.as_any().downcast_ref::().unwrap(); - // projection should directly wrap filter with no coalesce step - let _filter = projection - .input() - .as_any() - .downcast_ref::() - .unwrap(); + let _filter = plan.as_any().downcast_ref::().unwrap(); Ok(()) } diff --git a/datafusion/core/src/physical_plan/mod.rs b/datafusion/core/src/physical_plan/mod.rs index c8f5eb793910f..dbd1024ae482d 100644 --- a/datafusion/core/src/physical_plan/mod.rs +++ b/datafusion/core/src/physical_plan/mod.rs @@ -318,15 +318,14 @@ pub fn with_new_children_if_necessary( /// let normalized = Path::from_filesystem_path(working_directory).unwrap(); /// let plan_string = plan_string.replace(normalized.as_ref(), "WORKING_DIR"); /// -/// assert_eq!("ProjectionExec: expr=[a@0 as a]\ -/// \n CoalesceBatchesExec: target_batch_size=8192\ -/// \n FilterExec: a@0 < 5\ -/// \n RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1\ -/// \n CsvExec: files={1 group: [[WORKING_DIR/tests/data/example.csv]]}, has_header=true, limit=None, projection=[a]", +/// assert_eq!("CoalesceBatchesExec: target_batch_size=8192\ +/// \n FilterExec: a@0 < 5\ +/// \n RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1\ +/// \n CsvExec: files={1 group: [[WORKING_DIR/tests/data/example.csv]]}, has_header=true, limit=None, projection=[a]", /// plan_string.trim()); /// /// let one_line = format!("{}", displayable_plan.one_line()); -/// assert_eq!("ProjectionExec: expr=[a@0 as a]", one_line.trim()); +/// assert_eq!("CoalesceBatchesExec: target_batch_size=8192", one_line.trim()); /// } /// ``` /// diff --git a/datafusion/core/tests/custom_sources.rs b/datafusion/core/tests/custom_sources.rs index 9842f1b596e73..e78351500ca9c 100644 --- a/datafusion/core/tests/custom_sources.rs +++ b/datafusion/core/tests/custom_sources.rs @@ -22,7 +22,7 @@ use arrow::record_batch::RecordBatch; use datafusion::execution::context::{SessionContext, SessionState, TaskContext}; use datafusion::from_slice::FromSlice; use datafusion::logical_expr::{ - col, Expr, LogicalPlan, LogicalPlanBuilder, Projection, TableScan, UNNAMED_TABLE, + col, Expr, LogicalPlan, LogicalPlanBuilder, TableScan, UNNAMED_TABLE, }; use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::expressions::PhysicalSortExpr; @@ -214,24 +214,18 @@ async fn custom_source_dataframe() -> Result<()> { let optimized_plan = state.optimize(&logical_plan)?; match &optimized_plan { - LogicalPlan::Projection(Projection { input, .. }) => match &**input { - LogicalPlan::TableScan(TableScan { - source, - projected_schema, - .. - }) => { - assert_eq!(source.schema().fields().len(), 2); - assert_eq!(projected_schema.fields().len(), 1); - } - _ => panic!("input to projection should be TableScan"), - }, - _ => panic!("expect optimized_plan to be projection"), + LogicalPlan::TableScan(TableScan { + source, + projected_schema, + .. + }) => { + assert_eq!(source.schema().fields().len(), 2); + assert_eq!(projected_schema.fields().len(), 1); + } + _ => panic!("input to projection should be TableScan"), } - let expected = format!( - "Projection: {UNNAMED_TABLE}.c2\ - \n TableScan: {UNNAMED_TABLE} projection=[c2]" - ); + let expected = format!("TableScan: {UNNAMED_TABLE} projection=[c2]"); assert_eq!(format!("{optimized_plan:?}"), expected); let physical_plan = state.create_physical_plan(&optimized_plan).await?; @@ -242,7 +236,7 @@ async fn custom_source_dataframe() -> Result<()> { let batches = collect(physical_plan, state.task_ctx()).await?; let origin_rec_batch = TEST_CUSTOM_RECORD_BATCH!()?; assert_eq!(1, batches.len()); - assert_eq!(1, batches[0].num_columns()); + assert_eq!(2, batches[0].num_columns()); assert_eq!(origin_rec_batch.num_rows(), batches[0].num_rows()); Ok(()) @@ -270,8 +264,8 @@ async fn optimizers_catch_all_statistics() { let expected = RecordBatch::try_new( Arc::new(Schema::new(vec![ Field::new("COUNT(UInt8(1))", DataType::Int64, false), - Field::new("MIN(test.c1)", DataType::Int32, false), - Field::new("MAX(test.c1)", DataType::Int32, false), + Field::new("MIN(c1)", DataType::Int32, false), + Field::new("MAX(c1)", DataType::Int32, false), ])), vec![ Arc::new(Int64Array::from_slice([4])), diff --git a/datafusion/core/tests/dataframe.rs b/datafusion/core/tests/dataframe.rs index 73d2ce3ba4ef8..a9e28848cc0ef 100644 --- a/datafusion/core/tests/dataframe.rs +++ b/datafusion/core/tests/dataframe.rs @@ -250,7 +250,7 @@ async fn select_with_alias_overwrite() -> Result<()> { )?; let ctx = SessionContext::new(); - ctx.register_batch("t", batch).unwrap(); + ctx.register_batch("t", batch)?; let df = ctx .table("t") @@ -502,12 +502,11 @@ async fn right_semi_with_alias_filter() -> Result<()> { .select(vec![col("t2.a"), col("t2.b"), col("t2.c")])?; let optimized_plan = df.clone().into_optimized_plan()?; let expected = vec![ - "Projection: t2.a, t2.b, t2.c [a:UInt32, b:Utf8, c:Int32]", - " RightSemi Join: t1.a = t2.a [a:UInt32, b:Utf8, c:Int32]", - " Filter: t1.c > Int32(1) [a:UInt32, c:Int32]", - " TableScan: t1 projection=[a, c] [a:UInt32, c:Int32]", - " Filter: t2.c > Int32(1) [a:UInt32, b:Utf8, c:Int32]", - " TableScan: t2 projection=[a, b, c] [a:UInt32, b:Utf8, c:Int32]", + "RightSemi Join: t1.a = t2.a [a:UInt32, b:Utf8, c:Int32]", + " Filter: t1.c > Int32(1) [a:UInt32, c:Int32]", + " TableScan: t1 projection=[a, c] [a:UInt32, c:Int32]", + " Filter: t2.c > Int32(1) [a:UInt32, b:Utf8, c:Int32]", + " TableScan: t2 projection=[a, b, c] [a:UInt32, b:Utf8, c:Int32]", ]; let formatted = optimized_plan.display_indent_schema().to_string(); @@ -547,11 +546,10 @@ async fn right_anti_filter_push_down() -> Result<()> { .select(vec![col("t2.a"), col("t2.b"), col("t2.c")])?; let optimized_plan = df.clone().into_optimized_plan()?; let expected = vec![ - "Projection: t2.a, t2.b, t2.c [a:UInt32, b:Utf8, c:Int32]", - " RightAnti Join: t1.a = t2.a Filter: t2.c > Int32(1) [a:UInt32, b:Utf8, c:Int32]", - " Filter: t1.c > Int32(1) [a:UInt32, c:Int32]", - " TableScan: t1 projection=[a, c] [a:UInt32, c:Int32]", - " TableScan: t2 projection=[a, b, c] [a:UInt32, b:Utf8, c:Int32]", + "RightAnti Join: t1.a = t2.a Filter: t2.c > Int32(1) [a:UInt32, b:Utf8, c:Int32]", + " Filter: t1.c > Int32(1) [a:UInt32, c:Int32]", + " TableScan: t1 projection=[a, c] [a:UInt32, c:Int32]", + " TableScan: t2 projection=[a, b, c] [a:UInt32, b:Utf8, c:Int32]", ]; let formatted = optimized_plan.display_indent_schema().to_string(); diff --git a/datafusion/core/tests/sql/aggregates.rs b/datafusion/core/tests/sql/aggregates.rs index 3eaa2fd5d1416..972c8a94a2f25 100644 --- a/datafusion/core/tests/sql/aggregates.rs +++ b/datafusion/core/tests/sql/aggregates.rs @@ -99,11 +99,11 @@ async fn aggregate_timestamps_count() -> Result<()> { .await; let expected = vec![ - "+----------------+-----------------+-----------------+---------------+", - "| COUNT(t.nanos) | COUNT(t.micros) | COUNT(t.millis) | COUNT(t.secs) |", - "+----------------+-----------------+-----------------+---------------+", - "| 3 | 3 | 3 | 3 |", - "+----------------+-----------------+-----------------+---------------+", + "+--------------+---------------+---------------+-------------+", + "| COUNT(nanos) | COUNT(micros) | COUNT(millis) | COUNT(secs) |", + "+--------------+---------------+---------------+-------------+", + "| 3 | 3 | 3 | 3 |", + "+--------------+---------------+---------------+-------------+", ]; assert_batches_sorted_eq!(expected, &results); @@ -185,11 +185,11 @@ async fn aggregate_times_count() -> Result<()> { .await; let expected = vec![ - "+----------------+-----------------+-----------------+---------------+", - "| COUNT(t.nanos) | COUNT(t.micros) | COUNT(t.millis) | COUNT(t.secs) |", - "+----------------+-----------------+-----------------+---------------+", - "| 4 | 4 | 4 | 4 |", - "+----------------+-----------------+-----------------+---------------+", + "+--------------+---------------+---------------+-------------+", + "| COUNT(nanos) | COUNT(micros) | COUNT(millis) | COUNT(secs) |", + "+--------------+---------------+---------------+-------------+", + "| 4 | 4 | 4 | 4 |", + "+--------------+---------------+---------------+-------------+", ]; assert_batches_sorted_eq!(expected, &results); diff --git a/datafusion/core/tests/sql/avro.rs b/datafusion/core/tests/sql/avro.rs index d7bbf261cddcd..d933db067d6d2 100644 --- a/datafusion/core/tests/sql/avro.rs +++ b/datafusion/core/tests/sql/avro.rs @@ -140,18 +140,16 @@ async fn avro_explain() { let expected = vec![ vec![ "logical_plan", - "Projection: COUNT(UInt8(1))\ - \n Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1))]]\ - \n TableScan: alltypes_plain projection=[id]", + "Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1))]]\ + \n TableScan: alltypes_plain projection=[id]", ], vec![ "physical_plan", - "ProjectionExec: expr=[COUNT(UInt8(1))@0 as COUNT(UInt8(1))]\ - \n AggregateExec: mode=Final, gby=[], aggr=[COUNT(UInt8(1))]\ - \n CoalescePartitionsExec\ - \n AggregateExec: mode=Partial, gby=[], aggr=[COUNT(UInt8(1))]\ - \n RepartitionExec: partitioning=RoundRobinBatch(NUM_CORES), input_partitions=1\ - \n AvroExec: files={1 group: [[ARROW_TEST_DATA/avro/alltypes_plain.avro]]}, limit=None\ + "AggregateExec: mode=Final, gby=[], aggr=[COUNT(UInt8(1))]\ + \n CoalescePartitionsExec\ + \n AggregateExec: mode=Partial, gby=[], aggr=[COUNT(UInt8(1))]\ + \n RepartitionExec: partitioning=RoundRobinBatch(NUM_CORES), input_partitions=1\ + \n AvroExec: files={1 group: [[ARROW_TEST_DATA/avro/alltypes_plain.avro]]}, limit=None\ \n", ], ]; diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 99f5bd2d670dc..7391f29de9214 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -757,10 +757,9 @@ async fn explain_logical_plan_only() { let expected = vec![ vec![ "logical_plan", - "Projection: COUNT(UInt8(1))\ - \n Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1))]]\ - \n SubqueryAlias: t\ - \n Values: (Utf8(\"a\"), Int64(1), Int64(100)), (Utf8(\"a\"), Int64(2), Int64(150))", + "Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1))]]\ + \n SubqueryAlias: t\ + \n Values: (Utf8(\"a\"), Int64(1), Int64(100)), (Utf8(\"a\"), Int64(2), Int64(150))", ]]; assert_eq!(expected, actual); } @@ -776,9 +775,8 @@ async fn explain_physical_plan_only() { let expected = vec![vec![ "physical_plan", - "ProjectionExec: expr=[COUNT(UInt8(1))@0 as COUNT(UInt8(1))]\ - \n ProjectionExec: expr=[2 as COUNT(UInt8(1))]\ - \n EmptyExec: produce_one_row=true\ + "ProjectionExec: expr=[2 as COUNT(UInt8(1))]\ + \n EmptyExec: produce_one_row=true\ \n", ]]; assert_eq!(expected, actual); diff --git a/datafusion/core/tests/sql/joins.rs b/datafusion/core/tests/sql/joins.rs index a32d959f56ccd..5675a4bd6c5f5 100644 --- a/datafusion/core/tests/sql/joins.rs +++ b/datafusion/core/tests/sql/joins.rs @@ -1354,10 +1354,9 @@ async fn hash_join_with_date32() -> Result<()> { let plan = dataframe.into_optimized_plan()?; let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.c1, t1.c2, t1.c3, t1.c4, t2.c1, t2.c2, t2.c3, t2.c4 [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N, c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", - " Inner Join: t1.c1 = t2.c1 [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N, c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", - " TableScan: t1 projection=[c1, c2, c3, c4] [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N]", - " TableScan: t2 projection=[c1, c2, c3, c4] [c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", + " Inner Join: t1.c1 = t2.c1 [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N, c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", + " TableScan: t1 projection=[c1, c2, c3, c4] [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N]", + " TableScan: t2 projection=[c1, c2, c3, c4] [c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -1392,10 +1391,9 @@ async fn hash_join_with_date64() -> Result<()> { let plan = dataframe.into_optimized_plan()?; let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.c1, t1.c2, t1.c3, t1.c4, t2.c1, t2.c2, t2.c3, t2.c4 [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N, c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", - " Left Join: t1.c2 = t2.c2 [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N, c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", - " TableScan: t1 projection=[c1, c2, c3, c4] [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N]", - " TableScan: t2 projection=[c1, c2, c3, c4] [c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", + " Left Join: t1.c2 = t2.c2 [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N, c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", + " TableScan: t1 projection=[c1, c2, c3, c4] [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N]", + " TableScan: t2 projection=[c1, c2, c3, c4] [c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -1432,10 +1430,9 @@ async fn hash_join_with_decimal() -> Result<()> { let plan = dataframe.into_optimized_plan()?; let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.c1, t1.c2, t1.c3, t1.c4, t2.c1, t2.c2, t2.c3, t2.c4 [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N, c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", - " Right Join: CAST(t1.c3 AS Decimal128(10, 2)) = t2.c3 [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N, c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", - " TableScan: t1 projection=[c1, c2, c3, c4] [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N]", - " TableScan: t2 projection=[c1, c2, c3, c4] [c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", + " Right Join: CAST(t1.c3 AS Decimal128(10, 2)) = t2.c3 [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N, c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", + " TableScan: t1 projection=[c1, c2, c3, c4] [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N]", + " TableScan: t2 projection=[c1, c2, c3, c4] [c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -1472,10 +1469,9 @@ async fn hash_join_with_dictionary() -> Result<()> { let plan = dataframe.into_optimized_plan()?; let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.c1, t1.c2, t1.c3, t1.c4, t2.c1, t2.c2, t2.c3, t2.c4 [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N, c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", - " Inner Join: t1.c4 = t2.c4 [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N, c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", - " TableScan: t1 projection=[c1, c2, c3, c4] [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N]", - " TableScan: t2 projection=[c1, c2, c3, c4] [c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", + " Inner Join: t1.c4 = t2.c4 [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N, c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", + " TableScan: t1 projection=[c1, c2, c3, c4] [c1:Date32;N, c2:Date64;N, c3:Decimal128(5, 2);N, c4:Dictionary(Int32, Utf8);N]", + " TableScan: t2 projection=[c1, c2, c3, c4] [c1:Date32;N, c2:Date64;N, c3:Decimal128(10, 2);N, c4:Dictionary(Int32, Utf8);N]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -1512,12 +1508,11 @@ async fn reduce_left_join_1() -> Result<()> { let plan = dataframe.into_optimized_plan()?; let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int, t2.t2_id, t2.t2_name, t2.t2_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " Inner Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " Filter: t1.t1_id < UInt32(100) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " Filter: t2.t2_id < UInt32(100) [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " Inner Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " Filter: t1.t1_id < UInt32(100) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " Filter: t2.t2_id < UInt32(100) [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -1560,12 +1555,11 @@ async fn reduce_left_join_2() -> Result<()> { let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int, t2.t2_id, t2.t2_name, t2.t2_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " Filter: t2.t2_int < UInt32(10) OR t1.t1_int > UInt32(2) AND t2.t2_name != Utf8(\"w\") [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " Inner Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " Filter: t2.t2_int < UInt32(10) OR t2.t2_name != Utf8(\"w\") [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " Filter: t2.t2_int < UInt32(10) OR t1.t1_int > UInt32(2) AND t2.t2_name != Utf8(\"w\") [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " Inner Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " Filter: t2.t2_int < UInt32(10) OR t2.t2_name != Utf8(\"w\") [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -1603,16 +1597,15 @@ async fn reduce_left_join_3() -> Result<()> { let plan = dataframe.into_optimized_plan()?; let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t3.t1_id, t3.t1_name, t3.t1_int, t2.t2_id, t2.t2_name, t2.t2_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " Left Join: t3.t1_int = t2.t2_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " SubqueryAlias: t3 [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " Inner Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_int:UInt32;N]", - " Filter: t1.t1_id < UInt32(100) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " Filter: t2.t2_int < UInt32(3) AND t2.t2_id < UInt32(100) [t2_id:UInt32;N, t2_int:UInt32;N]", - " TableScan: t2 projection=[t2_id, t2_int] [t2_id:UInt32;N, t2_int:UInt32;N]", - " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " Left Join: t3.t1_int = t2.t2_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " SubqueryAlias: t3 [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " Projection: t1.t1_id, t1.t1_name, t1.t1_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " Inner Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_int:UInt32;N]", + " Filter: t1.t1_id < UInt32(100) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " Filter: t2.t2_int < UInt32(3) AND t2.t2_id < UInt32(100) [t2_id:UInt32;N, t2_int:UInt32;N]", + " TableScan: t2 projection=[t2_id, t2_int] [t2_id:UInt32;N, t2_int:UInt32;N]", + " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -1648,11 +1641,10 @@ async fn reduce_right_join_1() -> Result<()> { let plan = dataframe.into_optimized_plan()?; let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int, t2.t2_id, t2.t2_name, t2.t2_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " Inner Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " Filter: t1.t1_int IS NOT NULL [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " Inner Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " Filter: t1.t1_int IS NOT NULL [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -1690,11 +1682,10 @@ async fn reduce_right_join_2() -> Result<()> { let plan = dataframe.into_optimized_plan()?; let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int, t2.t2_id, t2.t2_name, t2.t2_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " Filter: t1.t1_int != t2.t2_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " Inner Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " Filter: t1.t1_int != t2.t2_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " Inner Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -1732,11 +1723,10 @@ async fn reduce_full_join_to_right_join() -> Result<()> { let plan = dataframe.into_optimized_plan()?; let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int, t2.t2_id, t2.t2_name, t2.t2_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " Right Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " Filter: t2.t2_name IS NOT NULL [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " Right Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " Filter: t2.t2_name IS NOT NULL [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -1776,11 +1766,10 @@ async fn reduce_full_join_to_left_join() -> Result<()> { let plan = dataframe.into_optimized_plan()?; let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int, t2.t2_id, t2.t2_name, t2.t2_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " Left Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " Filter: t1.t1_name != Utf8(\"b\") [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " Left Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " Filter: t1.t1_name != Utf8(\"b\") [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -1817,12 +1806,11 @@ async fn reduce_full_join_to_inner_join() -> Result<()> { let plan = dataframe.into_optimized_plan()?; let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int, t2.t2_id, t2.t2_name, t2.t2_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " Inner Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " Filter: t1.t1_name != Utf8(\"b\") [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " Filter: t2.t2_name = Utf8(\"x\") [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " Inner Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " Filter: t1.t1_name != Utf8(\"b\") [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " Filter: t2.t2_name = Utf8(\"x\") [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -1879,18 +1867,17 @@ async fn sort_merge_join_on_date32() -> Result<()> { let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let expected = vec![ - "ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, c1@4 as c1, c2@5 as c2, c3@6 as c3, c4@7 as c4]", - " SortMergeJoin: join_type=Inner, on=[(Column { name: \"c1\", index: 0 }, Column { name: \"c1\", index: 0 })]", - " SortExec: expr=[c1@0 ASC]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 2), input_partitions=2", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1]", - " SortExec: expr=[c1@0 ASC]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 2), input_partitions=2", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1]", + "SortMergeJoin: join_type=Inner, on=[(Column { name: \"c1\", index: 0 }, Column { name: \"c1\", index: 0 })]", + " SortExec: expr=[c1@0 ASC]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 2), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1]", + " SortExec: expr=[c1@0 ASC]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 2), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1]", ]; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -1924,20 +1911,19 @@ async fn sort_merge_join_on_decimal() -> Result<()> { let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let expected = vec![ - "ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, c1@4 as c1, c2@5 as c2, c3@6 as c3, c4@7 as c4]", - " ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, c1@5 as c1, c2@6 as c2, c3@7 as c3, c4@8 as c4]", - " SortMergeJoin: join_type=Right, on=[(Column { name: \"CAST(t1.c3 AS Decimal128(10, 2))\", index: 4 }, Column { name: \"c3\", index: 2 })]", - " SortExec: expr=[CAST(t1.c3 AS Decimal128(10, 2))@4 ASC]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([Column { name: \"CAST(t1.c3 AS Decimal128(10, 2))\", index: 4 }], 2), input_partitions=2", - " ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, CAST(c3@2 AS Decimal128(10, 2)) as CAST(t1.c3 AS Decimal128(10, 2))]", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1]", - " SortExec: expr=[c3@2 ASC]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([Column { name: \"c3\", index: 2 }], 2), input_partitions=2", + "ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, c1@5 as c1, c2@6 as c2, c3@7 as c3, c4@8 as c4]", + " SortMergeJoin: join_type=Right, on=[(Column { name: \"CAST(t1.c3 AS Decimal128(10, 2))\", index: 4 }, Column { name: \"c3\", index: 2 })]", + " SortExec: expr=[CAST(t1.c3 AS Decimal128(10, 2))@4 ASC]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([Column { name: \"CAST(t1.c3 AS Decimal128(10, 2))\", index: 4 }], 2), input_partitions=2", + " ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, CAST(c3@2 AS Decimal128(10, 2)) as CAST(t1.c3 AS Decimal128(10, 2))]", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", " MemoryExec: partitions=1, partition_sizes=[1]", + " SortExec: expr=[c3@2 ASC]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([Column { name: \"c3\", index: 2 }], 2), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1]", ]; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -1982,28 +1968,26 @@ async fn left_semi_join() -> Result<()> { vec![ "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", " SortExec: expr=[t1_id@0 ASC NULLS LAST]", - " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name]", - " CoalesceBatchesExec: target_batch_size=4096", - " HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: \"t1_id\", index: 0 }, Column { name: \"t2_id\", index: 0 })]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([Column { name: \"t1_id\", index: 0 }], 2), input_partitions=2", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " CoalesceBatchesExec: target_batch_size=4096", + " HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: \"t1_id\", index: 0 }, Column { name: \"t2_id\", index: 0 })]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([Column { name: \"t1_id\", index: 0 }], 2), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([Column { name: \"t2_id\", index: 0 }], 2), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " ProjectionExec: expr=[t2_id@0 as t2_id]", " MemoryExec: partitions=1, partition_sizes=[1]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([Column { name: \"t2_id\", index: 0 }], 2), input_partitions=2", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " ProjectionExec: expr=[t2_id@0 as t2_id]", - " MemoryExec: partitions=1, partition_sizes=[1]", ] } else { vec![ "SortExec: expr=[t1_id@0 ASC NULLS LAST]", - " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name]", - " CoalesceBatchesExec: target_batch_size=4096", - " HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(Column { name: \"t1_id\", index: 0 }, Column { name: \"t2_id\", index: 0 })]", + " CoalesceBatchesExec: target_batch_size=4096", + " HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(Column { name: \"t1_id\", index: 0 }, Column { name: \"t2_id\", index: 0 })]", + " MemoryExec: partitions=1, partition_sizes=[1]", + " ProjectionExec: expr=[t2_id@0 as t2_id]", " MemoryExec: partitions=1, partition_sizes=[1]", - " ProjectionExec: expr=[t2_id@0 as t2_id]", - " MemoryExec: partitions=1, partition_sizes=[1]", ] }; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); @@ -2062,26 +2046,24 @@ async fn left_semi_join() -> Result<()> { vec![ "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", " SortExec: expr=[t1_id@0 ASC NULLS LAST]", - " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name]", - " CoalesceBatchesExec: target_batch_size=4096", - " HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: \"t1_id\", index: 0 }, Column { name: \"t2_id\", index: 0 })]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([Column { name: \"t1_id\", index: 0 }], 2), input_partitions=2", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([Column { name: \"t2_id\", index: 0 }], 2), input_partitions=2", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1]", + " CoalesceBatchesExec: target_batch_size=4096", + " HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: \"t1_id\", index: 0 }, Column { name: \"t2_id\", index: 0 })]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([Column { name: \"t1_id\", index: 0 }], 2), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([Column { name: \"t2_id\", index: 0 }], 2), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1]", ] } else { vec![ "SortExec: expr=[t1_id@0 ASC NULLS LAST]", - " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name]", - " CoalesceBatchesExec: target_batch_size=4096", - " HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(Column { name: \"t1_id\", index: 0 }, Column { name: \"t2_id\", index: 0 })]", - " MemoryExec: partitions=1, partition_sizes=[1]", - " MemoryExec: partitions=1, partition_sizes=[1]", + " CoalesceBatchesExec: target_batch_size=4096", + " HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(Column { name: \"t1_id\", index: 0 }, Column { name: \"t2_id\", index: 0 })]", + " MemoryExec: partitions=1, partition_sizes=[1]", + " MemoryExec: partitions=1, partition_sizes=[1]", ] }; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); @@ -2120,11 +2102,10 @@ async fn left_semi_join_pushdown() -> Result<()> { let plan = dataframe.into_optimized_plan()?; let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name [t1_id:UInt32;N, t1_name:Utf8;N]", - " LeftSemi Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N]", - " TableScan: t1 projection=[t1_id, t1_name] [t1_id:UInt32;N, t1_name:Utf8;N]", - " Filter: t2.t2_int > UInt32(1) [t2_id:UInt32;N, t2_int:UInt32;N]", - " TableScan: t2 projection=[t2_id, t2_int] [t2_id:UInt32;N, t2_int:UInt32;N]", + " LeftSemi Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N]", + " TableScan: t1 projection=[t1_id, t1_name] [t1_id:UInt32;N, t1_name:Utf8;N]", + " Filter: t2.t2_int > UInt32(1) [t2_id:UInt32;N, t2_int:UInt32;N]", + " TableScan: t2 projection=[t2_id, t2_int] [t2_id:UInt32;N, t2_int:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -2255,29 +2236,25 @@ async fn right_semi_join() -> Result<()> { let physical_plan = dataframe.create_physical_plan().await?; let expected = if repartition_joins { vec!["SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", - " SortExec: expr=[t1_id@0 ASC NULLS LAST]", - " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int]", - " CoalesceBatchesExec: target_batch_size=4096", - " HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(Column { name: \"t2_id\", index: 0 }, Column { name: \"t1_id\", index: 0 })], filter=BinaryExpr { left: Column { name: \"t2_name\", index: 1 }, op: NotEq, right: Column { name: \"t1_name\", index: 0 } }", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([Column { name: \"t2_id\", index: 0 }], 2), input_partitions=2", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name]", - " MemoryExec: partitions=1, partition_sizes=[1]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([Column { name: \"t1_id\", index: 0 }], 2), input_partitions=2", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1]", + " SortExec: expr=[t1_id@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=4096", + " HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(Column { name: \"t2_id\", index: 0 }, Column { name: \"t1_id\", index: 0 })], filter=BinaryExpr { left: Column { name: \"t2_name\", index: 1 }, op: NotEq, right: Column { name: \"t1_name\", index: 0 } }", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([Column { name: \"t2_id\", index: 0 }], 2), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([Column { name: \"t1_id\", index: 0 }], 2), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1]", ] } else { vec![ "SortExec: expr=[t1_id@0 ASC NULLS LAST]", - " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int]", - " CoalesceBatchesExec: target_batch_size=4096", - " HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(Column { name: \"t2_id\", index: 0 }, Column { name: \"t1_id\", index: 0 })], filter=BinaryExpr { left: Column { name: \"t2_name\", index: 1 }, op: NotEq, right: Column { name: \"t1_name\", index: 0 } }", - " ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name]", - " MemoryExec: partitions=1, partition_sizes=[1]", - " MemoryExec: partitions=1, partition_sizes=[1]", + " CoalesceBatchesExec: target_batch_size=4096", + " HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(Column { name: \"t2_id\", index: 0 }, Column { name: \"t1_id\", index: 0 })], filter=BinaryExpr { left: Column { name: \"t2_name\", index: 1 }, op: NotEq, right: Column { name: \"t1_name\", index: 0 } }", + " MemoryExec: partitions=1, partition_sizes=[1]", + " MemoryExec: partitions=1, partition_sizes=[1]", ] }; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); @@ -2302,28 +2279,26 @@ async fn right_semi_join() -> Result<()> { let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let expected = if repartition_joins { - vec![ "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", - " SortExec: expr=[t1_id@0 ASC NULLS LAST]", - " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int]", - " CoalesceBatchesExec: target_batch_size=4096", - " HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(Column { name: \"t2_id\", index: 0 }, Column { name: \"t1_id\", index: 0 })], filter=BinaryExpr { left: Column { name: \"t2_name\", index: 0 }, op: NotEq, right: Column { name: \"t1_name\", index: 1 } }", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([Column { name: \"t2_id\", index: 0 }], 2), input_partitions=2", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([Column { name: \"t1_id\", index: 0 }], 2), input_partitions=2", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1]", + vec!["SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", + " SortExec: expr=[t1_id@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=4096", + " HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(Column { name: \"t2_id\", index: 0 }, Column { name: \"t1_id\", index: 0 })], filter=BinaryExpr { left: Column { name: \"t2_name\", index: 0 }, op: NotEq, right: Column { name: \"t1_name\", index: 1 } }", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([Column { name: \"t2_id\", index: 0 }], 2), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([Column { name: \"t1_id\", index: 0 }], 2), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1]", ] } else { vec![ "SortExec: expr=[t1_id@0 ASC NULLS LAST]", - " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int]", - " CoalesceBatchesExec: target_batch_size=4096", - " HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(Column { name: \"t2_id\", index: 0 }, Column { name: \"t1_id\", index: 0 })], filter=BinaryExpr { left: Column { name: \"t2_name\", index: 0 }, op: NotEq, right: Column { name: \"t1_name\", index: 1 } }", - " MemoryExec: partitions=1, partition_sizes=[1]", - " MemoryExec: partitions=1, partition_sizes=[1]", + " CoalesceBatchesExec: target_batch_size=4096", + " HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(Column { name: \"t2_id\", index: 0 }, Column { name: \"t1_id\", index: 0 })], filter=BinaryExpr { left: Column { name: \"t2_name\", index: 0 }, op: NotEq, right: Column { name: \"t1_name\", index: 1 } }", + " MemoryExec: partitions=1, partition_sizes=[1]", + " MemoryExec: partitions=1, partition_sizes=[1]", ] }; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); @@ -2385,10 +2360,9 @@ async fn reduce_cross_join_with_expr_join_key_all() -> Result<()> { let plan = dataframe.into_optimized_plan()?; let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int, t2.t2_id, t2.t2_name, t2.t2_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " Inner Join: CAST(t1.t1_id AS Int64) + Int64(12) = CAST(t2.t2_id AS Int64) + Int64(1) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " Inner Join: CAST(t1.t1_id AS Int64) + Int64(12) = CAST(t2.t2_id AS Int64) + Int64(1) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); @@ -2769,29 +2743,27 @@ async fn select_wildcard_with_expr_key_inner_join() -> Result<()> { let expected = if repartition_joins { vec![ "ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int]", - " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int]", - " CoalesceBatchesExec: target_batch_size=4096", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: \"t1_id\", index: 0 }, Column { name: \"t2.t2_id - UInt32(11)\", index: 3 })]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([Column { name: \"t1_id\", index: 0 }], 2), input_partitions=2", + " CoalesceBatchesExec: target_batch_size=4096", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: \"t1_id\", index: 0 }, Column { name: \"t2.t2_id - UInt32(11)\", index: 3 })]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([Column { name: \"t1_id\", index: 0 }], 2), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([Column { name: \"t2.t2_id - UInt32(11)\", index: 3 }], 2), input_partitions=2", + " ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as t2.t2_id - UInt32(11)]", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", " MemoryExec: partitions=1, partition_sizes=[1]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([Column { name: \"t2.t2_id - UInt32(11)\", index: 3 }], 2), input_partitions=2", - " ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as t2.t2_id - UInt32(11)]", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1]", - ] + ] } else { vec![ "ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int]", - " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int]", - " CoalesceBatchesExec: target_batch_size=4096", - " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(Column { name: \"t1_id\", index: 0 }, Column { name: \"t2.t2_id - UInt32(11)\", index: 3 })]", - " MemoryExec: partitions=1, partition_sizes=[1]", - " ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as t2.t2_id - UInt32(11)]", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1]", + " CoalesceBatchesExec: target_batch_size=4096", + " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(Column { name: \"t1_id\", index: 0 }, Column { name: \"t2.t2_id - UInt32(11)\", index: 3 })]", + " MemoryExec: partitions=1, partition_sizes=[1]", + " ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as t2.t2_id - UInt32(11)]", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1]", ] }; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); @@ -2831,10 +2803,9 @@ async fn join_with_type_coercion_for_equi_expr() -> Result<()> { let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t2_id:UInt32;N]", - " Inner Join: CAST(t1.t1_id AS Int64) + Int64(11) = CAST(t2.t2_id AS Int64) [t1_id:UInt32;N, t1_name:Utf8;N, t2_id:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name] [t1_id:UInt32;N, t1_name:Utf8;N]", - " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", + " Inner Join: CAST(t1.t1_id AS Int64) + Int64(11) = CAST(t2.t2_id AS Int64) [t1_id:UInt32;N, t1_name:Utf8;N, t2_id:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name] [t1_id:UInt32;N, t1_name:Utf8;N]", + " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); @@ -2873,10 +2844,9 @@ async fn join_only_with_filter() -> Result<()> { let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t2_id:UInt32;N]", - " Inner Join: Filter: CAST(t1.t1_id AS Int64) * Int64(4) < CAST(t2.t2_id AS Int64) [t1_id:UInt32;N, t1_name:Utf8;N, t2_id:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name] [t1_id:UInt32;N, t1_name:Utf8;N]", - " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", + " Inner Join: Filter: CAST(t1.t1_id AS Int64) * Int64(4) < CAST(t2.t2_id AS Int64) [t1_id:UInt32;N, t1_name:Utf8;N, t2_id:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name] [t1_id:UInt32;N, t1_name:Utf8;N]", + " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); @@ -2916,10 +2886,9 @@ async fn type_coercion_join_with_filter_and_equi_expr() -> Result<()> { let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t2_id:UInt32;N]", - " Inner Join: CAST(t1.t1_id AS Int64) * Int64(5) = CAST(t2.t2_id AS Int64) Filter: CAST(t1.t1_id AS Int64) * Int64(4) < CAST(t2.t2_id AS Int64) [t1_id:UInt32;N, t1_name:Utf8;N, t2_id:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name] [t1_id:UInt32;N, t1_name:Utf8;N]", - " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", + " Inner Join: CAST(t1.t1_id AS Int64) * Int64(5) = CAST(t2.t2_id AS Int64) Filter: CAST(t1.t1_id AS Int64) * Int64(4) < CAST(t2.t2_id AS Int64) [t1_id:UInt32;N, t1_name:Utf8;N, t2_id:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name] [t1_id:UInt32;N, t1_name:Utf8;N]", + " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); @@ -3014,12 +2983,11 @@ async fn subquery_to_join_with_both_side_expr() -> Result<()> { let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " LeftSemi Join: CAST(t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.CAST(t2_id AS Int64) + Int64(1) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " SubqueryAlias: __correlated_sq_1 [CAST(t2_id AS Int64) + Int64(1):Int64;N]", - " Projection: CAST(t2.t2_id AS Int64) + Int64(1) AS CAST(t2_id AS Int64) + Int64(1) [CAST(t2_id AS Int64) + Int64(1):Int64;N]", - " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", + " LeftSemi Join: CAST(t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.CAST(t2_id AS Int64) + Int64(1) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " SubqueryAlias: __correlated_sq_1 [CAST(t2_id AS Int64) + Int64(1):Int64;N]", + " Projection: CAST(t2.t2_id AS Int64) + Int64(1) AS CAST(t2_id AS Int64) + Int64(1) [CAST(t2_id AS Int64) + Int64(1):Int64;N]", + " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); @@ -3049,7 +3017,7 @@ async fn subquery_to_join_with_both_side_expr() -> Result<()> { async fn subquery_to_join_with_muti_filter() -> Result<()> { let ctx = create_join_context("t1_id", "t2_id", false)?; - let sql = "select t1.t1_id, t1.t1_name, t1.t1_int from t1 where t1.t1_id + 12 in + let sql = "select t1.t1_id, t1.t1_name, t1.t1_int from t1 where t1.t1_id + 12 in (select t2.t2_id + 1 from t2 where t1.t1_int <= t2.t2_int and t2.t2_int > 0)"; // assert logical plan @@ -3059,13 +3027,12 @@ async fn subquery_to_join_with_muti_filter() -> Result<()> { let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " LeftSemi Join: CAST(t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.CAST(t2_id AS Int64) + Int64(1) Filter: t1.t1_int <= __correlated_sq_1.t2_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " SubqueryAlias: __correlated_sq_1 [CAST(t2_id AS Int64) + Int64(1):Int64;N, t2_int:UInt32;N]", - " Projection: CAST(t2.t2_id AS Int64) + Int64(1) AS CAST(t2_id AS Int64) + Int64(1), t2.t2_int [CAST(t2_id AS Int64) + Int64(1):Int64;N, t2_int:UInt32;N]", - " Filter: t2.t2_int > UInt32(0) [t2_id:UInt32;N, t2_int:UInt32;N]", - " TableScan: t2 projection=[t2_id, t2_int] [t2_id:UInt32;N, t2_int:UInt32;N]", + " LeftSemi Join: CAST(t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.CAST(t2_id AS Int64) + Int64(1) Filter: t1.t1_int <= __correlated_sq_1.t2_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " SubqueryAlias: __correlated_sq_1 [CAST(t2_id AS Int64) + Int64(1):Int64;N, t2_int:UInt32;N]", + " Projection: CAST(t2.t2_id AS Int64) + Int64(1) AS CAST(t2_id AS Int64) + Int64(1), t2.t2_int [CAST(t2_id AS Int64) + Int64(1):Int64;N, t2_int:UInt32;N]", + " Filter: t2.t2_int > UInt32(0) [t2_id:UInt32;N, t2_int:UInt32;N]", + " TableScan: t2 projection=[t2_id, t2_int] [t2_id:UInt32;N, t2_int:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); @@ -3094,7 +3061,7 @@ async fn subquery_to_join_with_muti_filter() -> Result<()> { async fn three_projection_exprs_subquery_to_join() -> Result<()> { let ctx = create_join_context("t1_id", "t2_id", false)?; - let sql = "select t1.t1_id, t1.t1_name, t1.t1_int from t1 where t1.t1_id + 12 in + let sql = "select t1.t1_id, t1.t1_name, t1.t1_int from t1 where t1.t1_id + 12 in (select t2.t2_id + 1 from t2 where t1.t1_int <= t2.t2_int and t1.t1_name != t2.t2_name and t2.t2_int > 0)"; // assert logical plan @@ -3104,13 +3071,12 @@ async fn three_projection_exprs_subquery_to_join() -> Result<()> { let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " LeftSemi Join: CAST(t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.CAST(t2_id AS Int64) + Int64(1) Filter: t1.t1_int <= __correlated_sq_1.t2_int AND t1.t1_name != __correlated_sq_1.t2_name [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " SubqueryAlias: __correlated_sq_1 [CAST(t2_id AS Int64) + Int64(1):Int64;N, t2_int:UInt32;N, t2_name:Utf8;N]", - " Projection: CAST(t2.t2_id AS Int64) + Int64(1) AS CAST(t2_id AS Int64) + Int64(1), t2.t2_int, t2.t2_name [CAST(t2_id AS Int64) + Int64(1):Int64;N, t2_int:UInt32;N, t2_name:Utf8;N]", - " Filter: t2.t2_int > UInt32(0) [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", - " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " LeftSemi Join: CAST(t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.CAST(t2_id AS Int64) + Int64(1) Filter: t1.t1_int <= __correlated_sq_1.t2_int AND t1.t1_name != __correlated_sq_1.t2_name [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " SubqueryAlias: __correlated_sq_1 [CAST(t2_id AS Int64) + Int64(1):Int64;N, t2_int:UInt32;N, t2_name:Utf8;N]", + " Projection: CAST(t2.t2_id AS Int64) + Int64(1) AS CAST(t2_id AS Int64) + Int64(1), t2.t2_int, t2.t2_name [CAST(t2_id AS Int64) + Int64(1):Int64;N, t2_int:UInt32;N, t2_name:Utf8;N]", + " Filter: t2.t2_int > UInt32(0) [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); @@ -3139,7 +3105,7 @@ async fn three_projection_exprs_subquery_to_join() -> Result<()> { async fn in_subquery_to_join_with_correlated_outer_filter() -> Result<()> { let ctx = create_join_context("t1_id", "t2_id", false)?; - let sql = "select t1.t1_id, t1.t1_name, t1.t1_int from t1 where t1.t1_id + 12 in + let sql = "select t1.t1_id, t1.t1_name, t1.t1_int from t1 where t1.t1_id + 12 in (select t2.t2_id + 1 from t2 where t1.t1_int > 0)"; // assert logical plan @@ -3148,13 +3114,12 @@ async fn in_subquery_to_join_with_correlated_outer_filter() -> Result<()> { let plan = dataframe.into_optimized_plan().unwrap(); let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " LeftSemi Join: CAST(t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.CAST(t2_id AS Int64) + Int64(1) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " Filter: t1.t1_int > UInt32(0) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " SubqueryAlias: __correlated_sq_1 [CAST(t2_id AS Int64) + Int64(1):Int64;N]", - " Projection: CAST(t2.t2_id AS Int64) + Int64(1) AS CAST(t2_id AS Int64) + Int64(1) [CAST(t2_id AS Int64) + Int64(1):Int64;N]", - " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", + " LeftSemi Join: CAST(t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.CAST(t2_id AS Int64) + Int64(1) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " Filter: t1.t1_int > UInt32(0) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " SubqueryAlias: __correlated_sq_1 [CAST(t2_id AS Int64) + Int64(1):Int64;N]", + " Projection: CAST(t2.t2_id AS Int64) + Int64(1) AS CAST(t2_id AS Int64) + Int64(1) [CAST(t2_id AS Int64) + Int64(1):Int64;N]", + " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); @@ -3170,7 +3135,7 @@ async fn in_subquery_to_join_with_correlated_outer_filter() -> Result<()> { async fn not_in_subquery_to_join_with_correlated_outer_filter() -> Result<()> { let ctx = create_join_context("t1_id", "t2_id", false)?; - let sql = "select t1.t1_id, t1.t1_name, t1.t1_int from t1 where t1.t1_id + 12 not in + let sql = "select t1.t1_id, t1.t1_name, t1.t1_int from t1 where t1.t1_id + 12 not in (select t2.t2_id + 1 from t2 where t1.t1_int > 0)"; // assert logical plan @@ -3179,12 +3144,11 @@ async fn not_in_subquery_to_join_with_correlated_outer_filter() -> Result<()> { let plan = dataframe.into_optimized_plan().unwrap(); let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " LeftAnti Join: CAST(t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.CAST(t2_id AS Int64) + Int64(1) Filter: t1.t1_int > UInt32(0) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " SubqueryAlias: __correlated_sq_1 [CAST(t2_id AS Int64) + Int64(1):Int64;N]", - " Projection: CAST(t2.t2_id AS Int64) + Int64(1) AS CAST(t2_id AS Int64) + Int64(1) [CAST(t2_id AS Int64) + Int64(1):Int64;N]", - " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", + " LeftAnti Join: CAST(t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.CAST(t2_id AS Int64) + Int64(1) Filter: t1.t1_int > UInt32(0) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " SubqueryAlias: __correlated_sq_1 [CAST(t2_id AS Int64) + Int64(1):Int64;N]", + " Projection: CAST(t2.t2_id AS Int64) + Int64(1) AS CAST(t2_id AS Int64) + Int64(1) [CAST(t2_id AS Int64) + Int64(1):Int64;N]", + " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); @@ -3200,7 +3164,7 @@ async fn not_in_subquery_to_join_with_correlated_outer_filter() -> Result<()> { async fn in_subquery_to_join_with_outer_filter() -> Result<()> { let ctx = create_join_context("t1_id", "t2_id", false)?; - let sql = "select t1.t1_id, t1.t1_name, t1.t1_int from t1 where t1.t1_id + 12 in + let sql = "select t1.t1_id, t1.t1_name, t1.t1_int from t1 where t1.t1_id + 12 in (select t2.t2_id + 1 from t2 where t1.t1_int <= t2.t2_int and t1.t1_name != t2.t2_name) and t1.t1_id > 0"; // assert logical plan @@ -3210,13 +3174,12 @@ async fn in_subquery_to_join_with_outer_filter() -> Result<()> { let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " LeftSemi Join: CAST(t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.CAST(t2_id AS Int64) + Int64(1) Filter: t1.t1_int <= __correlated_sq_1.t2_int AND t1.t1_name != __correlated_sq_1.t2_name [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " Filter: t1.t1_id > UInt32(0) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " SubqueryAlias: __correlated_sq_1 [CAST(t2_id AS Int64) + Int64(1):Int64;N, t2_int:UInt32;N, t2_name:Utf8;N]", - " Projection: CAST(t2.t2_id AS Int64) + Int64(1) AS CAST(t2_id AS Int64) + Int64(1), t2.t2_int, t2.t2_name [CAST(t2_id AS Int64) + Int64(1):Int64;N, t2_int:UInt32;N, t2_name:Utf8;N]", - " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", + " LeftSemi Join: CAST(t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.CAST(t2_id AS Int64) + Int64(1) Filter: t1.t1_int <= __correlated_sq_1.t2_int AND t1.t1_name != __correlated_sq_1.t2_name [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " Filter: t1.t1_id > UInt32(0) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " SubqueryAlias: __correlated_sq_1 [CAST(t2_id AS Int64) + Int64(1):Int64;N, t2_int:UInt32;N, t2_name:Utf8;N]", + " Projection: CAST(t2.t2_id AS Int64) + Int64(1) AS CAST(t2_id AS Int64) + Int64(1), t2.t2_int, t2.t2_name [CAST(t2_id AS Int64) + Int64(1):Int64;N, t2_int:UInt32;N, t2_name:Utf8;N]", + " TableScan: t2 projection=[t2_id, t2_name, t2_int] [t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); @@ -3245,8 +3208,8 @@ async fn in_subquery_to_join_with_outer_filter() -> Result<()> { async fn two_in_subquery_to_join_with_outer_filter() -> Result<()> { let ctx = create_join_context("t1_id", "t2_id", false)?; - let sql = "select t1.t1_id, t1.t1_name, t1.t1_int from t1 where t1.t1_id + 12 in - (select t2.t2_id + 1 from t2) + let sql = "select t1.t1_id, t1.t1_name, t1.t1_int from t1 where t1.t1_id + 12 in + (select t2.t2_id + 1 from t2) and t1.t1_int in(select t2.t2_int + 1 from t2) and t1.t1_id > 0"; @@ -3257,17 +3220,16 @@ async fn two_in_subquery_to_join_with_outer_filter() -> Result<()> { let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " LeftSemi Join: CAST(t1.t1_int AS Int64) = __correlated_sq_2.CAST(t2_int AS Int64) + Int64(1) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " LeftSemi Join: CAST(t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.CAST(t2_id AS Int64) + Int64(1) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " Filter: t1.t1_id > UInt32(0) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " SubqueryAlias: __correlated_sq_1 [CAST(t2_id AS Int64) + Int64(1):Int64;N]", - " Projection: CAST(t2.t2_id AS Int64) + Int64(1) AS CAST(t2_id AS Int64) + Int64(1) [CAST(t2_id AS Int64) + Int64(1):Int64;N]", - " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", - " SubqueryAlias: __correlated_sq_2 [CAST(t2_int AS Int64) + Int64(1):Int64;N]", - " Projection: CAST(t2.t2_int AS Int64) + Int64(1) AS CAST(t2_int AS Int64) + Int64(1) [CAST(t2_int AS Int64) + Int64(1):Int64;N]", - " TableScan: t2 projection=[t2_int] [t2_int:UInt32;N]", + " LeftSemi Join: CAST(t1.t1_int AS Int64) = __correlated_sq_2.CAST(t2_int AS Int64) + Int64(1) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " LeftSemi Join: CAST(t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.CAST(t2_id AS Int64) + Int64(1) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " Filter: t1.t1_id > UInt32(0) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " SubqueryAlias: __correlated_sq_1 [CAST(t2_id AS Int64) + Int64(1):Int64;N]", + " Projection: CAST(t2.t2_id AS Int64) + Int64(1) AS CAST(t2_id AS Int64) + Int64(1) [CAST(t2_id AS Int64) + Int64(1):Int64;N]", + " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", + " SubqueryAlias: __correlated_sq_2 [CAST(t2_int AS Int64) + Int64(1):Int64;N]", + " Projection: CAST(t2.t2_int AS Int64) + Int64(1) AS CAST(t2_int AS Int64) + Int64(1) [CAST(t2_int AS Int64) + Int64(1):Int64;N]", + " TableScan: t2 projection=[t2_int] [t2_int:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); @@ -3296,8 +3258,8 @@ async fn right_as_inner_table_nested_loop_join() -> Result<()> { let ctx = create_nested_loop_join_context()?; // Distribution: left is `UnspecifiedDistribution`, right is `SinglePartition`. - let sql = "SELECT t1.t1_id, t2.t2_id - FROM t1 INNER JOIN t2 ON t1.t1_id > t2.t2_id + let sql = "SELECT t1.t1_id, t2.t2_id + FROM t1 INNER JOIN t2 ON t1.t1_id > t2.t2_id WHERE t1.t1_id > 10 AND t2.t2_int > 1"; let msg = format!("Creating logical plan for '{sql}'"); @@ -3346,8 +3308,8 @@ async fn left_as_inner_table_nested_loop_join() -> Result<()> { let ctx = create_nested_loop_join_context()?; // Distribution: left is `SinglePartition`, right is `UnspecifiedDistribution`. - let sql = "SELECT t1.t1_id,t2.t2_id FROM (select t1_id from t1 where t1.t1_id > 22) as t1 - RIGHT JOIN (select t2_id from t2 where t2.t2_id > 11) as t2 + let sql = "SELECT t1.t1_id,t2.t2_id FROM (select t1_id from t1 where t1.t1_id > 22) as t1 + RIGHT JOIN (select t2_id from t2 where t2.t2_id > 11) as t2 ON t1.t1_id < t2.t2_id"; let msg = format!("Creating logical plan for '{sql}'"); @@ -3356,19 +3318,16 @@ async fn left_as_inner_table_nested_loop_join() -> Result<()> { // left is single partition side, so it will be visited many times. let expected = vec![ - "ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@1 as t2_id]", - " NestedLoopJoinExec: join_type=Right, filter=BinaryExpr { left: Column { name: \"t1_id\", index: 0 }, op: Lt, right: Column { name: \"t2_id\", index: 1 } }", - " CoalescePartitionsExec", - " ProjectionExec: expr=[t1_id@0 as t1_id]", - " CoalesceBatchesExec: target_batch_size=4096", - " FilterExec: t1_id@0 > 22", - " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1]", - " ProjectionExec: expr=[t2_id@0 as t2_id]", - " CoalesceBatchesExec: target_batch_size=4096", - " FilterExec: t2_id@0 > 11", - " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1]", + "NestedLoopJoinExec: join_type=Right, filter=BinaryExpr { left: Column { name: \"t1_id\", index: 0 }, op: Lt, right: Column { name: \"t2_id\", index: 1 } }", + " CoalescePartitionsExec", + " CoalesceBatchesExec: target_batch_size=4096", + " FilterExec: t1_id@0 > 22", + " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1]", + " CoalesceBatchesExec: target_batch_size=4096", + " FilterExec: t2_id@0 > 11", + " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1]", ]; let formatted = displayable(physical_plan.as_ref()).indent().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -3409,11 +3368,9 @@ async fn exists_subquery_to_join_expr_filter() -> Result<()> { let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " LeftSemi Join: Filter: CAST(t1.t1_id AS Int64) + Int64(1) > CAST(t2.t2_id AS Int64) * Int64(2) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " Projection: t2.t2_id [t2_id:UInt32;N]", - " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", + " LeftSemi Join: Filter: CAST(t1.t1_id AS Int64) + Int64(1) > CAST(t2.t2_id AS Int64) * Int64(2) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -3453,12 +3410,11 @@ async fn exists_subquery_to_join_inner_filter() -> Result<()> { // `t2.t2_int < 3` will be kept in the subquery filter. let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " LeftSemi Join: Filter: CAST(t1.t1_id AS Int64) + Int64(1) > CAST(t2.t2_id AS Int64) * Int64(2) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " Projection: t2.t2_id [t2_id:UInt32;N]", - " Filter: t2.t2_int < UInt32(3) [t2_id:UInt32;N, t2_int:UInt32;N]", - " TableScan: t2 projection=[t2_id, t2_int] [t2_id:UInt32;N, t2_int:UInt32;N]", + " LeftSemi Join: Filter: CAST(t1.t1_id AS Int64) + Int64(1) > CAST(t2.t2_id AS Int64) * Int64(2) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " Projection: t2.t2_id [t2_id:UInt32;N]", + " Filter: t2.t2_int < UInt32(3) [t2_id:UInt32;N, t2_int:UInt32;N]", + " TableScan: t2 projection=[t2_id, t2_int] [t2_id:UInt32;N, t2_int:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -3496,12 +3452,10 @@ async fn exists_subquery_to_join_outer_filter() -> Result<()> { // `t1.t1_int < 3` will be moved to the filter of t1. let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " LeftSemi Join: Filter: CAST(t1.t1_id AS Int64) + Int64(1) > CAST(t2.t2_id AS Int64) * Int64(2) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " Filter: t1.t1_int < UInt32(3) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " Projection: t2.t2_id [t2_id:UInt32;N]", - " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", + " LeftSemi Join: Filter: CAST(t1.t1_id AS Int64) + Int64(1) > CAST(t2.t2_id AS Int64) * Int64(2) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " Filter: t1.t1_int < UInt32(3) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -3538,11 +3492,9 @@ async fn not_exists_subquery_to_join_expr_filter() -> Result<()> { let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", - " Projection: t1.t1_id, t1.t1_name, t1.t1_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " LeftAnti Join: Filter: CAST(t1.t1_id AS Int64) + Int64(1) > CAST(t2.t2_id AS Int64) * Int64(2) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", - " Projection: t2.t2_id [t2_id:UInt32;N]", - " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", + " LeftAnti Join: Filter: CAST(t1.t1_id AS Int64) + Int64(1) > CAST(t2.t2_id AS Int64) * Int64(2) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t1 projection=[t1_id, t1_name, t1_int] [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N]", + " TableScan: t2 projection=[t2_id] [t2_id:UInt32;N]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); diff --git a/datafusion/core/tests/sql/json.rs b/datafusion/core/tests/sql/json.rs index ad5a4c808f7e0..965a9c14fc985 100644 --- a/datafusion/core/tests/sql/json.rs +++ b/datafusion/core/tests/sql/json.rs @@ -83,18 +83,16 @@ async fn json_explain() { let expected = vec![ vec![ "logical_plan", - "Projection: COUNT(UInt8(1))\ - \n Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1))]]\ - \n TableScan: t1 projection=[a]", + "Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1))]]\ + \n TableScan: t1 projection=[a]", ], vec![ "physical_plan", - "ProjectionExec: expr=[COUNT(UInt8(1))@0 as COUNT(UInt8(1))]\ - \n AggregateExec: mode=Final, gby=[], aggr=[COUNT(UInt8(1))]\ - \n CoalescePartitionsExec\ - \n AggregateExec: mode=Partial, gby=[], aggr=[COUNT(UInt8(1))]\ - \n RepartitionExec: partitioning=RoundRobinBatch(NUM_CORES), input_partitions=1\ - \n JsonExec: limit=None, files={1 group: [[WORKING_DIR/tests/jsons/2.json]]}\n", + "AggregateExec: mode=Final, gby=[], aggr=[COUNT(UInt8(1))]\ + \n CoalescePartitionsExec\ + \n AggregateExec: mode=Partial, gby=[], aggr=[COUNT(UInt8(1))]\ + \n RepartitionExec: partitioning=RoundRobinBatch(NUM_CORES), input_partitions=1\ + \n JsonExec: limit=None, files={1 group: [[WORKING_DIR/tests/jsons/2.json]]}\n", ], ]; assert_eq!(expected, actual); diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index 48fd890a35057..d8b6a83f2f1a3 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -28,7 +28,7 @@ use chrono::Duration; use datafusion::config::ConfigOptions; use datafusion::datasource::TableProvider; use datafusion::from_slice::FromSlice; -use datafusion::logical_expr::{Aggregate, LogicalPlan, Projection, TableScan}; +use datafusion::logical_expr::{Aggregate, LogicalPlan, TableScan}; use datafusion::physical_plan::metrics::MetricValue; use datafusion::physical_plan::ExecutionPlan; use datafusion::physical_plan::ExecutionPlanVisitor; @@ -1588,18 +1588,15 @@ async fn nyc() -> Result<()> { let optimized_plan = dataframe.into_optimized_plan().unwrap(); match &optimized_plan { - LogicalPlan::Projection(Projection { input, .. }) => match input.as_ref() { - LogicalPlan::Aggregate(Aggregate { input, .. }) => match input.as_ref() { - LogicalPlan::TableScan(TableScan { - ref projected_schema, - .. - }) => { - assert_eq!(2, projected_schema.fields().len()); - assert_eq!(projected_schema.field(0).name(), "passenger_count"); - assert_eq!(projected_schema.field(1).name(), "fare_amount"); - } - _ => unreachable!(), - }, + LogicalPlan::Aggregate(Aggregate { input, .. }) => match input.as_ref() { + LogicalPlan::TableScan(TableScan { + ref projected_schema, + .. + }) => { + assert_eq!(2, projected_schema.fields().len()); + assert_eq!(projected_schema.field(0).name(), "passenger_count"); + assert_eq!(projected_schema.field(1).name(), "fare_amount"); + } _ => unreachable!(), }, _ => unreachable!(), diff --git a/datafusion/core/tests/sql/parquet.rs b/datafusion/core/tests/sql/parquet.rs index b01e8b843d52c..31cd0da21b936 100644 --- a/datafusion/core/tests/sql/parquet.rs +++ b/datafusion/core/tests/sql/parquet.rs @@ -309,11 +309,11 @@ async fn parquet_query_with_max_min() { let sql = "SELECT max(c1) FROM foo"; let actual = execute_to_batches(&ctx, sql).await; let expected = vec![ - "+-------------+", - "| MAX(foo.c1) |", - "+-------------+", - "| 3 |", - "+-------------+", + "+---------+", + "| MAX(c1) |", + "+---------+", + "| 3 |", + "+---------+", ]; assert_batches_eq!(expected, &actual); @@ -333,11 +333,11 @@ async fn parquet_query_with_max_min() { let sql = "SELECT max(c3) FROM foo"; let actual = execute_to_batches(&ctx, sql).await; let expected = vec![ - "+-------------+", - "| MAX(foo.c3) |", - "+-------------+", - "| 300 |", - "+-------------+", + "+---------+", + "| MAX(c3) |", + "+---------+", + "| 300 |", + "+---------+", ]; assert_batches_eq!(expected, &actual); diff --git a/datafusion/core/tests/sql/predicates.rs b/datafusion/core/tests/sql/predicates.rs index cd4e02111cda2..ac57e0a1f2241 100644 --- a/datafusion/core/tests/sql/predicates.rs +++ b/datafusion/core/tests/sql/predicates.rs @@ -156,15 +156,14 @@ where let plan = dataframe.into_optimized_plan().unwrap(); let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); - let expected =vec![ - "Projection: part.p_partkey, SUM(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(DISTINCT partsupp.ps_suppkey) [p_partkey:Int64, SUM(lineitem.l_extendedprice):Decimal128(25, 2);N, AVG(lineitem.l_discount):Decimal128(19, 6);N, COUNT(DISTINCT partsupp.ps_suppkey):Int64;N]", - " Aggregate: groupBy=[[part.p_partkey]], aggr=[[SUM(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(DISTINCT partsupp.ps_suppkey)]] [p_partkey:Int64, SUM(lineitem.l_extendedprice):Decimal128(25, 2);N, AVG(lineitem.l_discount):Decimal128(19, 6);N, COUNT(DISTINCT partsupp.ps_suppkey):Int64;N]", - " Inner Join: part.p_partkey = partsupp.ps_partkey [l_partkey:Int64, l_extendedprice:Decimal128(15, 2), l_discount:Decimal128(15, 2), p_partkey:Int64, p_brand:Utf8, ps_partkey:Int64, ps_suppkey:Int64]", - " Inner Join: lineitem.l_partkey = part.p_partkey [l_partkey:Int64, l_extendedprice:Decimal128(15, 2), l_discount:Decimal128(15, 2), p_partkey:Int64, p_brand:Utf8]", - " TableScan: lineitem projection=[l_partkey, l_extendedprice, l_discount] [l_partkey:Int64, l_extendedprice:Decimal128(15, 2), l_discount:Decimal128(15, 2)]", - " Filter: part.p_brand = Utf8(\"Brand#12\") OR part.p_brand = Utf8(\"Brand#23\") [p_partkey:Int64, p_brand:Utf8]", - " TableScan: part projection=[p_partkey, p_brand], partial_filters=[part.p_brand = Utf8(\"Brand#12\") OR part.p_brand = Utf8(\"Brand#23\")] [p_partkey:Int64, p_brand:Utf8]", - " TableScan: partsupp projection=[ps_partkey, ps_suppkey] [ps_partkey:Int64, ps_suppkey:Int64]", + let expected = vec![ + "Aggregate: groupBy=[[part.p_partkey]], aggr=[[SUM(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(DISTINCT partsupp.ps_suppkey)]] [p_partkey:Int64, SUM(lineitem.l_extendedprice):Decimal128(25, 2);N, AVG(lineitem.l_discount):Decimal128(19, 6);N, COUNT(DISTINCT partsupp.ps_suppkey):Int64;N]", + " Inner Join: part.p_partkey = partsupp.ps_partkey [l_partkey:Int64, l_extendedprice:Decimal128(15, 2), l_discount:Decimal128(15, 2), p_partkey:Int64, p_brand:Utf8, ps_partkey:Int64, ps_suppkey:Int64]", + " Inner Join: lineitem.l_partkey = part.p_partkey [l_partkey:Int64, l_extendedprice:Decimal128(15, 2), l_discount:Decimal128(15, 2), p_partkey:Int64, p_brand:Utf8]", + " TableScan: lineitem projection=[l_partkey, l_extendedprice, l_discount] [l_partkey:Int64, l_extendedprice:Decimal128(15, 2), l_discount:Decimal128(15, 2)]", + " Filter: part.p_brand = Utf8(\"Brand#12\") OR part.p_brand = Utf8(\"Brand#23\") [p_partkey:Int64, p_brand:Utf8]", + " TableScan: part projection=[p_partkey, p_brand], partial_filters=[part.p_brand = Utf8(\"Brand#12\") OR part.p_brand = Utf8(\"Brand#23\")] [p_partkey:Int64, p_brand:Utf8]", + " TableScan: partsupp projection=[ps_partkey, ps_suppkey] [ps_partkey:Int64, ps_suppkey:Int64]", ]; assert_eq!( diff --git a/datafusion/core/tests/sql/projection.rs b/datafusion/core/tests/sql/projection.rs index b4627c5979bfb..ac697b11768cf 100644 --- a/datafusion/core/tests/sql/projection.rs +++ b/datafusion/core/tests/sql/projection.rs @@ -175,22 +175,18 @@ async fn projection_on_table_scan() -> Result<()> { let state = ctx.state(); let optimized_plan = state.optimize(&logical_plan)?; match &optimized_plan { - LogicalPlan::Projection(Projection { input, .. }) => match &**input { - LogicalPlan::TableScan(TableScan { - source, - projected_schema, - .. - }) => { - assert_eq!(source.schema().fields().len(), 3); - assert_eq!(projected_schema.fields().len(), 1); - } - _ => panic!("input to projection should be TableScan"), - }, - _ => panic!("expect optimized_plan to be projection"), + LogicalPlan::TableScan(TableScan { + source, + projected_schema, + .. + }) => { + assert_eq!(source.schema().fields().len(), 3); + assert_eq!(projected_schema.fields().len(), 1); + } + _ => panic!("input to projection should be TableScan"), } - let expected = "Projection: test.c2\ - \n TableScan: test projection=[c2]"; + let expected = "TableScan: test projection=[c2]"; assert_eq!(format!("{optimized_plan:?}"), expected); let physical_plan = state.create_physical_plan(&optimized_plan).await?; @@ -291,24 +287,18 @@ async fn projection_on_memory_scan() -> Result<()> { let state = ctx.state(); let optimized_plan = state.optimize(&plan)?; match &optimized_plan { - LogicalPlan::Projection(Projection { input, .. }) => match &**input { - LogicalPlan::TableScan(TableScan { - source, - projected_schema, - .. - }) => { - assert_eq!(source.schema().fields().len(), 3); - assert_eq!(projected_schema.fields().len(), 1); - } - _ => panic!("input to projection should be InMemoryScan"), - }, - _ => panic!("expect optimized_plan to be projection"), + LogicalPlan::TableScan(TableScan { + source, + projected_schema, + .. + }) => { + assert_eq!(source.schema().fields().len(), 3); + assert_eq!(projected_schema.fields().len(), 1); + } + _ => panic!("input to projection should be InMemoryScan"), } - let expected = format!( - "Projection: {UNNAMED_TABLE}.b\ - \n TableScan: {UNNAMED_TABLE} projection=[b]" - ); + let expected = format!("TableScan: {UNNAMED_TABLE} projection=[b]"); assert_eq!(format!("{optimized_plan:?}"), expected); let physical_plan = state.create_physical_plan(&optimized_plan).await?; diff --git a/datafusion/core/tests/sql/select.rs b/datafusion/core/tests/sql/select.rs index 00cc06bb44da1..9a746f65694bc 100644 --- a/datafusion/core/tests/sql/select.rs +++ b/datafusion/core/tests/sql/select.rs @@ -763,11 +763,11 @@ async fn query_on_string_dictionary() -> Result<()> { let sql = "SELECT COUNT(d1) FROM test"; let actual = execute_to_batches(&ctx, sql).await; let expected = vec![ - "+----------------+", - "| COUNT(test.d1) |", - "+----------------+", - "| 2 |", - "+----------------+", + "+-----------+", + "| COUNT(d1) |", + "+-----------+", + "| 2 |", + "+-----------+", ]; assert_batches_eq!(expected, &actual); diff --git a/datafusion/core/tests/sql/union.rs b/datafusion/core/tests/sql/union.rs index c257bc304298d..4cf908aa85828 100644 --- a/datafusion/core/tests/sql/union.rs +++ b/datafusion/core/tests/sql/union.rs @@ -39,16 +39,12 @@ async fn union_with_except_input() -> Result<()> { " Union [name:UInt8;N]", " LeftAnti Join: t1.name = t2.name [name:UInt8;N]", " Aggregate: groupBy=[[t1.name]], aggr=[[]] [name:UInt8;N]", - " Projection: t1.name [name:UInt8;N]", - " TableScan: t1 projection=[name] [name:UInt8;N]", - " Projection: t2.name [name:UInt8;N]", - " TableScan: t2 projection=[name] [name:UInt8;N]", + " TableScan: t1 projection=[name] [name:UInt8;N]", + " TableScan: t2 projection=[name] [name:UInt8;N]", " LeftAnti Join: t2.name = t1.name [name:UInt8;N]", " Aggregate: groupBy=[[t2.name]], aggr=[[]] [name:UInt8;N]", - " Projection: t2.name [name:UInt8;N]", - " TableScan: t2 projection=[name] [name:UInt8;N]", - " Projection: t1.name [name:UInt8;N]", - " TableScan: t1 projection=[name] [name:UInt8;N]", + " TableScan: t2 projection=[name] [name:UInt8;N]", + " TableScan: t1 projection=[name] [name:UInt8;N]", ]; let formatted = plan.display_indent_schema().to_string(); @@ -82,17 +78,13 @@ async fn union_with_type_coercion() -> Result<()> { " Union [id:Int32;N, name:UInt8;N]", " LeftAnti Join: t1.id = CAST(t2.id AS Int32), t1.name = t2.name [id:Int32;N, name:UInt8;N]", " Aggregate: groupBy=[[t1.id, t1.name]], aggr=[[]] [id:Int32;N, name:UInt8;N]", - " Projection: t1.id, t1.name [id:Int32;N, name:UInt8;N]", - " TableScan: t1 projection=[id, name] [id:Int32;N, name:UInt8;N]", - " Projection: t2.id, t2.name [id:UInt8;N, name:UInt8;N]", - " TableScan: t2 projection=[id, name] [id:UInt8;N, name:UInt8;N]", + " TableScan: t1 projection=[id, name] [id:Int32;N, name:UInt8;N]", + " TableScan: t2 projection=[id, name] [id:UInt8;N, name:UInt8;N]", " Projection: CAST(t2.id AS Int32) AS id, t2.name [id:Int32;N, name:UInt8;N]", " LeftAnti Join: CAST(t2.id AS Int32) = t1.id, t2.name = t1.name [id:UInt8;N, name:UInt8;N]", " Aggregate: groupBy=[[t2.id, t2.name]], aggr=[[]] [id:UInt8;N, name:UInt8;N]", - " Projection: t2.id, t2.name [id:UInt8;N, name:UInt8;N]", - " TableScan: t2 projection=[id, name] [id:UInt8;N, name:UInt8;N]", - " Projection: t1.id, t1.name [id:Int32;N, name:UInt8;N]", - " TableScan: t1 projection=[id, name] [id:Int32;N, name:UInt8;N]", + " TableScan: t2 projection=[id, name] [id:UInt8;N, name:UInt8;N]", + " TableScan: t1 projection=[id, name] [id:Int32;N, name:UInt8;N]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 8f5d1584e0a9d..c60ee304dbdaf 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -1741,7 +1741,6 @@ async fn test_window_agg_with_global_limit() -> Result<()> { " AggregateExec: mode=Partial, gby=[], aggr=[ARRAYAGG(aggregate_test_100.c13)]", " GlobalLimitExec: skip=0, fetch=1", " SortExec: fetch=1, expr=[c13@0 ASC NULLS LAST]", - " ProjectionExec: expr=[c13@0 as c13]", ] }; diff --git a/datafusion/core/tests/sqllogictests/test_files/explain.slt b/datafusion/core/tests/sqllogictests/test_files/explain.slt index 9eca732c45f32..fe1d3ac2e4c47 100644 --- a/datafusion/core/tests/sqllogictests/test_files/explain.slt +++ b/datafusion/core/tests/sqllogictests/test_files/explain.slt @@ -59,9 +59,8 @@ query TT EXPLAIN select count(*) from (values ('a', 1, 100), ('a', 2, 150)) as t (c1,c2,c3) ---- physical_plan -ProjectionExec: expr=[COUNT(UInt8(1))@0 as COUNT(UInt8(1))] - ProjectionExec: expr=[2 as COUNT(UInt8(1))] - EmptyExec: produce_one_row=true +ProjectionExec: expr=[2 as COUNT(UInt8(1))] + EmptyExec: produce_one_row=true statement ok set datafusion.explain.physical_plan_only = false diff --git a/datafusion/core/tests/sqllogictests/test_files/window.slt b/datafusion/core/tests/sqllogictests/test_files/window.slt index 64920bb3dbb16..88be8adc71c08 100644 --- a/datafusion/core/tests/sqllogictests/test_files/window.slt +++ b/datafusion/core/tests/sqllogictests/test_files/window.slt @@ -258,18 +258,17 @@ Sort: d.b ASC NULLS LAST Aggregate: groupBy=[[d.b]], aggr=[[MAX(d.a)]] SubqueryAlias: d SubqueryAlias: _data2 - Projection: s.a, s.b - SubqueryAlias: s - SubqueryAlias: _sample_data - Union - Projection: Int64(1) AS a, Utf8("aa") AS b - EmptyRelation - Projection: Int64(3) AS a, Utf8("aa") AS b - EmptyRelation - Projection: Int64(5) AS a, Utf8("bb") AS b - EmptyRelation - Projection: Int64(7) AS a, Utf8("bb") AS b - EmptyRelation + SubqueryAlias: s + SubqueryAlias: _sample_data + Union + Projection: Int64(1) AS a, Utf8("aa") AS b + EmptyRelation + Projection: Int64(3) AS a, Utf8("aa") AS b + EmptyRelation + Projection: Int64(5) AS a, Utf8("bb") AS b + EmptyRelation + Projection: Int64(7) AS a, Utf8("bb") AS b + EmptyRelation physical_plan SortPreservingMergeExec: [b@0 ASC NULLS LAST] SortExec: expr=[b@0 ASC NULLS LAST] @@ -278,16 +277,15 @@ SortPreservingMergeExec: [b@0 ASC NULLS LAST] CoalesceBatchesExec: target_batch_size=8192 RepartitionExec: partitioning=Hash([Column { name: "b", index: 0 }], 4), input_partitions=4 AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[MAX(d.a)] - ProjectionExec: expr=[a@0 as a, b@1 as b] - UnionExec - ProjectionExec: expr=[1 as a, aa as b] - EmptyExec: produce_one_row=true - ProjectionExec: expr=[3 as a, aa as b] - EmptyExec: produce_one_row=true - ProjectionExec: expr=[5 as a, bb as b] - EmptyExec: produce_one_row=true - ProjectionExec: expr=[7 as a, bb as b] - EmptyExec: produce_one_row=true + UnionExec + ProjectionExec: expr=[1 as a, aa as b] + EmptyExec: produce_one_row=true + ProjectionExec: expr=[3 as a, aa as b] + EmptyExec: produce_one_row=true + ProjectionExec: expr=[5 as a, bb as b] + EmptyExec: produce_one_row=true + ProjectionExec: expr=[7 as a, bb as b] + EmptyExec: produce_one_row=true # Check actual result: query TI diff --git a/datafusion/core/tests/user_defined_plan.rs b/datafusion/core/tests/user_defined_plan.rs index af7eb61f46057..3b1ea76a84a26 100644 --- a/datafusion/core/tests/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined_plan.rs @@ -218,8 +218,7 @@ async fn topk_plan() -> Result<()> { let mut expected = vec![ "| logical_plan after topk | TopK: k=3 |", - "| | Projection: sales.customer_id, sales.revenue |", - "| | TableScan: sales projection=[customer_id,revenue] |", + "| | TableScan: sales projection=[customer_id,revenue] |", ].join("\n"); let explain_query = format!("EXPLAIN VERBOSE {QUERY}"); diff --git a/datafusion/optimizer/src/eliminate_project.rs b/datafusion/optimizer/src/eliminate_project.rs new file mode 100644 index 0000000000000..cf77a227f6287 --- /dev/null +++ b/datafusion/optimizer/src/eliminate_project.rs @@ -0,0 +1,94 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::optimizer::ApplyOrder; +use crate::{OptimizerConfig, OptimizerRule}; +use datafusion_common::{DFSchemaRef, Result}; +use datafusion_expr::logical_plan::LogicalPlan; +use datafusion_expr::{Expr, Projection}; + +/// Optimization rule that eliminate unnecessary [LogicalPlan::Projection]. +#[derive(Default)] +pub struct EliminateProjection; + +impl EliminateProjection { + #[allow(missing_docs)] + pub fn new() -> Self { + Self {} + } +} + +impl OptimizerRule for EliminateProjection { + fn try_optimize( + &self, + plan: &LogicalPlan, + _config: &dyn OptimizerConfig, + ) -> Result> { + match plan { + LogicalPlan::Projection(projection) => { + let child_plan = projection.input.as_ref(); + match child_plan { + LogicalPlan::Join(_) + | LogicalPlan::CrossJoin(_) + | LogicalPlan::Union(_) + | LogicalPlan::Filter(_) + | LogicalPlan::Sort(_) => { + if can_eliminate(projection, child_plan.schema()) { + Ok(Some(child_plan.clone())) + } else { + Ok(None) + } + } + _ => { + if plan.schema() == child_plan.schema() { + Ok(Some(child_plan.clone())) + } else { + Ok(None) + } + } + } + } + _ => Ok(None), + } + } + + fn name(&self) -> &str { + "eliminate_projection" + } + + fn apply_order(&self) -> Option { + Some(ApplyOrder::TopDown) + } +} + +fn can_eliminate(projection: &Projection, schema: &DFSchemaRef) -> bool { + if projection.expr.len() != schema.fields().len() { + return false; + } + for (i, e) in projection.expr.iter().enumerate() { + match e { + Expr::Column(c) => { + let d = schema.fields().get(i).unwrap(); + if c != &d.qualified_column() && c != &d.unqualified_column() { + return false; + } + } + _ => return false, + } + } + true +} diff --git a/datafusion/optimizer/src/lib.rs b/datafusion/optimizer/src/lib.rs index 5de1c55ede7e6..ca0611d4e17e7 100644 --- a/datafusion/optimizer/src/lib.rs +++ b/datafusion/optimizer/src/lib.rs @@ -23,6 +23,7 @@ pub mod eliminate_cross_join; pub mod eliminate_filter; pub mod eliminate_limit; pub mod eliminate_outer_join; +pub mod eliminate_project; pub mod extract_equijoin_predicate; pub mod filter_null_join_keys; pub mod inline_table_scan; diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 0e8ca3991b8e9..2b27b6702d3c7 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -24,6 +24,7 @@ use crate::eliminate_cross_join::EliminateCrossJoin; use crate::eliminate_filter::EliminateFilter; use crate::eliminate_limit::EliminateLimit; use crate::eliminate_outer_join::EliminateOuterJoin; +use crate::eliminate_project::EliminateProjection; use crate::extract_equijoin_predicate::ExtractEquijoinPredicate; use crate::filter_null_join_keys::FilterNullJoinKeys; use crate::inline_table_scan::InlineTableScan; @@ -237,6 +238,7 @@ impl Optimizer { Arc::new(UnwrapCastInComparison::new()), Arc::new(CommonSubexprEliminate::new()), Arc::new(PushDownProjection::new()), + Arc::new(EliminateProjection::new()), ]; Self::with_rules(rules) diff --git a/datafusion/optimizer/tests/integration-test.rs b/datafusion/optimizer/tests/integration-test.rs index c79f3e9dc0bc5..b84d15fae735a 100644 --- a/datafusion/optimizer/tests/integration-test.rs +++ b/datafusion/optimizer/tests/integration-test.rs @@ -105,8 +105,7 @@ fn distribute_by() -> Result<()> { let sql = "SELECT col_int32, col_utf8 FROM test DISTRIBUTE BY (col_utf8)"; let plan = test_sql(sql)?; let expected = "Repartition: DistributeBy(col_utf8)\ - \n Projection: test.col_int32, test.col_utf8\ - \n TableScan: test projection=[col_int32, col_utf8]"; + \n TableScan: test projection=[col_int32, col_utf8]"; assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -121,9 +120,8 @@ fn semi_join_with_join_filter() -> Result<()> { let expected = "Projection: test.col_utf8\ \n LeftSemi Join: test.col_int32 = t2.col_int32 Filter: test.col_uint32 != t2.col_uint32\ \n TableScan: test projection=[col_int32, col_uint32, col_utf8]\ - \n Projection: t2.col_int32, t2.col_uint32\ - \n SubqueryAlias: t2\ - \n TableScan: test projection=[col_int32, col_uint32]"; + \n SubqueryAlias: t2\ + \n TableScan: test projection=[col_int32, col_uint32]"; assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -136,11 +134,10 @@ fn anti_join_with_join_filter() -> Result<()> { AND test.col_uint32 != t2.col_uint32)"; let plan = test_sql(sql)?; let expected = "Projection: test.col_utf8\ - \n LeftAnti Join: test.col_int32 = t2.col_int32 Filter: test.col_uint32 != t2.col_uint32\ - \n TableScan: test projection=[col_int32, col_uint32, col_utf8]\ - \n Projection: t2.col_int32, t2.col_uint32\ - \n SubqueryAlias: t2\ - \n TableScan: test projection=[col_int32, col_uint32]"; + \n LeftAnti Join: test.col_int32 = t2.col_int32 Filter: test.col_uint32 != t2.col_uint32\ + \n TableScan: test projection=[col_int32, col_uint32, col_utf8]\ + \n SubqueryAlias: t2\ + \n TableScan: test projection=[col_int32, col_uint32]"; assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -151,12 +148,10 @@ fn where_exists_distinct() -> Result<()> { let sql = "SELECT col_int32 FROM test WHERE EXISTS (\ SELECT DISTINCT col_int32 FROM test t2 WHERE test.col_int32 = t2.col_int32)"; let plan = test_sql(sql)?; - let expected = "Projection: test.col_int32\ - \n LeftSemi Join: test.col_int32 = t2.col_int32\ - \n TableScan: test projection=[col_int32]\ - \n Projection: t2.col_int32\ - \n SubqueryAlias: t2\ - \n TableScan: test projection=[col_int32]"; + let expected = "LeftSemi Join: test.col_int32 = t2.col_int32\ + \n TableScan: test projection=[col_int32]\ + \n SubqueryAlias: t2\ + \n TableScan: test projection=[col_int32]"; assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -185,9 +180,9 @@ fn between_date32_plus_interval() -> Result<()> { WHERE col_date32 between '1998-03-18' AND cast('1998-03-18' as date) + INTERVAL '90 days'"; let plan = test_sql(sql)?; let expected = - "Projection: COUNT(Int64(1))\n Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]]\ - \n Filter: test.col_date32 >= Date32(\"10303\") AND test.col_date32 <= Date32(\"10393\")\ - \n TableScan: test projection=[col_date32]"; + "Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]]\ + \n Filter: test.col_date32 >= Date32(\"10303\") AND test.col_date32 <= Date32(\"10393\")\ + \n TableScan: test projection=[col_date32]"; assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -198,9 +193,9 @@ fn between_date64_plus_interval() -> Result<()> { WHERE col_date64 between '1998-03-18T00:00:00' AND cast('1998-03-18' as date) + INTERVAL '90 days'"; let plan = test_sql(sql)?; let expected = - "Projection: COUNT(Int64(1))\n Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]]\ - \n Filter: test.col_date64 >= Date64(\"890179200000\") AND test.col_date64 <= Date64(\"897955200000\")\ - \n TableScan: test projection=[col_date64]"; + "Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]]\ + \n Filter: test.col_date64 >= Date64(\"890179200000\") AND test.col_date64 <= Date64(\"897955200000\")\ + \n TableScan: test projection=[col_date64]"; assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -276,15 +271,15 @@ fn propagate_empty_relation() { fn join_keys_in_subquery_alias() { let sql = "SELECT * FROM test AS A, ( SELECT col_int32 as key FROM test ) AS B where A.col_int32 = B.key;"; let plan = test_sql(sql).unwrap(); - let expected = "Projection: a.col_int32, a.col_uint32, a.col_utf8, a.col_date32, a.col_date64, a.col_ts_nano_none, a.col_ts_nano_utc, b.key\ - \n Inner Join: a.col_int32 = b.key\ - \n SubqueryAlias: a\ + let expected = "Inner Join: a.col_int32 = b.key\ + \n SubqueryAlias: a\ + \n Filter: test.col_int32 IS NOT NULL\ + \n TableScan: test projection=[col_int32, col_uint32, col_utf8, col_date32, col_date64, col_ts_nano_none, col_ts_nano_utc]\ + \n SubqueryAlias: b\ + \n Projection: test.col_int32 AS key\ \n Filter: test.col_int32 IS NOT NULL\ - \n TableScan: test projection=[col_int32, col_uint32, col_utf8, col_date32, col_date64, col_ts_nano_none, col_ts_nano_utc]\ - \n SubqueryAlias: b\ - \n Projection: test.col_int32 AS key\ - \n Filter: test.col_int32 IS NOT NULL\ - \n TableScan: test projection=[col_int32]"; + \n TableScan: test projection=[col_int32]"; + assert_eq!(expected, format!("{plan:?}")); } @@ -292,19 +287,18 @@ fn join_keys_in_subquery_alias() { fn join_keys_in_subquery_alias_1() { let sql = "SELECT * FROM test AS A, ( SELECT test.col_int32 AS key FROM test JOIN test AS C on test.col_int32 = C.col_int32 ) AS B where A.col_int32 = B.key;"; let plan = test_sql(sql).unwrap(); - let expected = "Projection: a.col_int32, a.col_uint32, a.col_utf8, a.col_date32, a.col_date64, a.col_ts_nano_none, a.col_ts_nano_utc, b.key\ - \n Inner Join: a.col_int32 = b.key\ - \n SubqueryAlias: a\ - \n Filter: test.col_int32 IS NOT NULL\ - \n TableScan: test projection=[col_int32, col_uint32, col_utf8, col_date32, col_date64, col_ts_nano_none, col_ts_nano_utc]\ - \n SubqueryAlias: b\ - \n Projection: test.col_int32 AS key\ - \n Inner Join: test.col_int32 = c.col_int32\ + let expected = "Inner Join: a.col_int32 = b.key\ + \n SubqueryAlias: a\ + \n Filter: test.col_int32 IS NOT NULL\ + \n TableScan: test projection=[col_int32, col_uint32, col_utf8, col_date32, col_date64, col_ts_nano_none, col_ts_nano_utc]\ + \n SubqueryAlias: b\ + \n Projection: test.col_int32 AS key\ + \n Inner Join: test.col_int32 = c.col_int32\ + \n Filter: test.col_int32 IS NOT NULL\ + \n TableScan: test projection=[col_int32]\ + \n SubqueryAlias: c\ \n Filter: test.col_int32 IS NOT NULL\ - \n TableScan: test projection=[col_int32]\ - \n SubqueryAlias: c\ - \n Filter: test.col_int32 IS NOT NULL\ - \n TableScan: test projection=[col_int32]"; + \n TableScan: test projection=[col_int32]"; assert_eq!(expected, format!("{plan:?}")); } @@ -329,9 +323,8 @@ fn test_same_name_but_not_ambiguous() { \n Projection: t1.col_int32 AS col_int32\ \n SubqueryAlias: t1\ \n TableScan: test projection=[col_int32]\ - \n Projection: t2.col_int32\ - \n SubqueryAlias: t2\ - \n TableScan: test projection=[col_int32]"; + \n SubqueryAlias: t2\ + \n TableScan: test projection=[col_int32]"; assert_eq!(expected, format!("{plan:?}")); } diff --git a/datafusion/substrait/tests/roundtrip_logical_plan.rs b/datafusion/substrait/tests/roundtrip_logical_plan.rs index 005c37e16c39f..47ce1bbcd9b63 100644 --- a/datafusion/substrait/tests/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/roundtrip_logical_plan.rs @@ -178,9 +178,8 @@ mod tests { async fn aggregate_case() -> Result<()> { assert_expected_plan( "SELECT SUM(CASE WHEN a > 0 THEN 1 ELSE NULL END) FROM data", - "Projection: SUM(CASE WHEN data.a > Int64(0) THEN Int64(1) ELSE Int64(NULL) END)\ - \n Aggregate: groupBy=[[]], aggr=[[SUM(CASE WHEN data.a > Int64(0) THEN Int64(1) ELSE Int64(NULL) END)]]\ - \n TableScan: data projection=[a]", + "Aggregate: groupBy=[[]], aggr=[[SUM(CASE WHEN data.a > Int64(0) THEN Int64(1) ELSE Int64(NULL) END)]]\ + \n TableScan: data projection=[a]", ) .await } @@ -227,14 +226,11 @@ mod tests { async fn simple_intersect() -> Result<()> { assert_expected_plan( "SELECT COUNT(*) FROM (SELECT data.a FROM data INTERSECT SELECT data2.a FROM data2);", - "Projection: COUNT(Int16(1))\ - \n Aggregate: groupBy=[[]], aggr=[[COUNT(Int16(1))]]\ - \n LeftSemi Join: data.a = data2.a\ - \n Aggregate: groupBy=[[data.a]], aggr=[[]]\ - \n Projection: data.a\ - \n TableScan: data projection=[a]\ - \n Projection: data2.a\ - \n TableScan: data2 projection=[a]", + "Aggregate: groupBy=[[]], aggr=[[COUNT(Int16(1))]]\ + \n LeftSemi Join: data.a = data2.a\ + \n Aggregate: groupBy=[[data.a]], aggr=[[]]\ + \n TableScan: data projection=[a]\ + \n TableScan: data2 projection=[a]", ) .await }