diff --git a/benchmarks/expected-plans/q10.txt b/benchmarks/expected-plans/q10.txt index 25189feb61e1..153159539691 100644 --- a/benchmarks/expected-plans/q10.txt +++ b/benchmarks/expected-plans/q10.txt @@ -1,12 +1,17 @@ Sort: revenue DESC NULLS FIRST Projection: customer.c_custkey, customer.c_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue, customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment Aggregate: groupBy=[[customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment]], aggr=[[SUM(CAST(lineitem.l_extendedprice AS Decimal128(38, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(38, 4))) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] - Inner Join: customer.c_nationkey = nation.n_nationkey - Inner Join: orders.o_orderkey = lineitem.l_orderkey - Inner Join: customer.c_custkey = orders.o_custkey - TableScan: customer projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment] - Filter: orders.o_orderdate >= Date32("8674") AND orders.o_orderdate < Date32("8766") - TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate] - Filter: lineitem.l_returnflag = Utf8("R") - TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag] - TableScan: nation projection=[n_nationkey, n_name] \ No newline at end of file + Projection: customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name + Inner Join: customer.c_nationkey = nation.n_nationkey + Projection: customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount + Inner Join: orders.o_orderkey = lineitem.l_orderkey + Projection: customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey + Inner Join: customer.c_custkey = orders.o_custkey + TableScan: customer projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment] + Projection: orders.o_orderkey, orders.o_custkey + Filter: orders.o_orderdate >= Date32("8674") AND orders.o_orderdate < Date32("8766") + TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate] + Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount + Filter: lineitem.l_returnflag = Utf8("R") + TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag] + TableScan: nation projection=[n_nationkey, n_name] \ No newline at end of file diff --git a/benchmarks/expected-plans/q11.txt b/benchmarks/expected-plans/q11.txt index 7d8e145487ba..adcba5967229 100644 --- a/benchmarks/expected-plans/q11.txt +++ b/benchmarks/expected-plans/q11.txt @@ -3,18 +3,24 @@ Sort: value DESC NULLS FIRST Filter: CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty) AS Decimal128(38, 15)) > CAST(__scalar_sq_1.__value AS Decimal128(38, 15)) CrossJoin: Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[SUM(CAST(partsupp.ps_supplycost AS Decimal128(26, 2)) * CAST(partsupp.ps_availqty AS Decimal128(26, 2)))]] - Inner Join: supplier.s_nationkey = nation.n_nationkey - Inner Join: partsupp.ps_suppkey = supplier.s_suppkey - TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost] - TableScan: supplier projection=[s_suppkey, s_nationkey] - Filter: nation.n_name = Utf8("GERMANY") - TableScan: nation projection=[n_nationkey, n_name] - SubqueryAlias: __scalar_sq_1 - Projection: CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty) AS Float64) * Float64(0.0001) AS __value - Aggregate: groupBy=[[]], aggr=[[SUM(CAST(partsupp.ps_supplycost AS Decimal128(26, 2)) * CAST(partsupp.ps_availqty AS Decimal128(26, 2)))]] - Inner Join: supplier.s_nationkey = nation.n_nationkey + Projection: partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost + Inner Join: supplier.s_nationkey = nation.n_nationkey + Projection: partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey Inner Join: partsupp.ps_suppkey = supplier.s_suppkey - TableScan: partsupp projection=[ps_suppkey, ps_availqty, ps_supplycost] + TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost] TableScan: supplier projection=[s_suppkey, s_nationkey] + Projection: nation.n_nationkey Filter: nation.n_name = Utf8("GERMANY") - TableScan: nation projection=[n_nationkey, n_name] \ No newline at end of file + TableScan: nation projection=[n_nationkey, n_name] + SubqueryAlias: __scalar_sq_1 + Projection: CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty) AS Float64) * Float64(0.0001) AS __value + Aggregate: groupBy=[[]], aggr=[[SUM(CAST(partsupp.ps_supplycost AS Decimal128(26, 2)) * CAST(partsupp.ps_availqty AS Decimal128(26, 2)))]] + Projection: partsupp.ps_availqty, partsupp.ps_supplycost + Inner Join: supplier.s_nationkey = nation.n_nationkey + Projection: partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey + Inner Join: partsupp.ps_suppkey = supplier.s_suppkey + TableScan: partsupp projection=[ps_suppkey, ps_availqty, ps_supplycost] + TableScan: supplier projection=[s_suppkey, s_nationkey] + Projection: nation.n_nationkey + Filter: nation.n_name = Utf8("GERMANY") + TableScan: nation projection=[n_nationkey, n_name] \ No newline at end of file diff --git a/benchmarks/expected-plans/q12.txt b/benchmarks/expected-plans/q12.txt index 547d12e9f19a..8b35840ce918 100644 --- a/benchmarks/expected-plans/q12.txt +++ b/benchmarks/expected-plans/q12.txt @@ -1,7 +1,9 @@ Sort: lineitem.l_shipmode ASC NULLS LAST Projection: lineitem.l_shipmode, SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END) AS high_line_count, SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END) AS low_line_count Aggregate: groupBy=[[lineitem.l_shipmode]], aggr=[[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)]] - Inner Join: lineitem.l_orderkey = orders.o_orderkey - Filter: (lineitem.l_shipmode = Utf8("SHIP") OR lineitem.l_shipmode = Utf8("MAIL")) AND lineitem.l_commitdate < lineitem.l_receiptdate AND lineitem.l_shipdate < lineitem.l_commitdate AND lineitem.l_receiptdate >= Date32("8766") AND lineitem.l_receiptdate < Date32("9131") - TableScan: lineitem projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode] - TableScan: orders projection=[o_orderkey, o_orderpriority] \ No newline at end of file + Projection: lineitem.l_shipmode, orders.o_orderpriority + Inner Join: lineitem.l_orderkey = orders.o_orderkey + Projection: lineitem.l_orderkey, lineitem.l_shipmode + Filter: (lineitem.l_shipmode = Utf8("SHIP") OR lineitem.l_shipmode = Utf8("MAIL")) AND lineitem.l_commitdate < lineitem.l_receiptdate AND lineitem.l_shipdate < lineitem.l_commitdate AND lineitem.l_receiptdate >= Date32("8766") AND lineitem.l_receiptdate < Date32("9131") + TableScan: lineitem projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode] + TableScan: orders projection=[o_orderkey, o_orderpriority] \ No newline at end of file diff --git a/benchmarks/expected-plans/q13.txt b/benchmarks/expected-plans/q13.txt index e2c4e9774e2c..045b75758f2c 100644 --- a/benchmarks/expected-plans/q13.txt +++ b/benchmarks/expected-plans/q13.txt @@ -5,7 +5,9 @@ Sort: custdist DESC NULLS FIRST, c_count DESC NULLS FIRST SubqueryAlias: c_orders Projection: COUNT(orders.o_orderkey) Aggregate: groupBy=[[customer.c_custkey]], aggr=[[COUNT(orders.o_orderkey)]] - Left Join: customer.c_custkey = orders.o_custkey - TableScan: customer projection=[c_custkey] - Filter: orders.o_comment NOT LIKE Utf8("%special%requests%") - TableScan: orders projection=[o_orderkey, o_custkey, o_comment] \ No newline at end of file + Projection: customer.c_custkey, orders.o_orderkey + Left Join: customer.c_custkey = orders.o_custkey + TableScan: customer projection=[c_custkey] + Projection: orders.o_orderkey, orders.o_custkey + Filter: orders.o_comment NOT LIKE Utf8("%special%requests%") + TableScan: orders projection=[o_orderkey, o_custkey, o_comment] \ No newline at end of file diff --git a/benchmarks/expected-plans/q14.txt b/benchmarks/expected-plans/q14.txt index edafe4608210..089259a7f083 100644 --- a/benchmarks/expected-plans/q14.txt +++ b/benchmarks/expected-plans/q14.txt @@ -2,6 +2,7 @@ Projection: Float64(100) * CAST(SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") T Aggregate: groupBy=[[]], aggr=[[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN CAST(lineitem.l_extendedprice AS Decimal128(38, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(38, 4))CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(38, 4))Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2))CAST(lineitem.l_discount AS Decimal128(23, 2))lineitem.l_discountDecimal128(Some(100),23,2)CAST(lineitem.l_extendedprice AS Decimal128(38, 4))lineitem.l_extendedprice AS lineitem.l_extendedprice * Decimal128(Some(100),23,2) - lineitem.l_discount ELSE Decimal128(Some(0),38,4) END) AS SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(CAST(lineitem.l_extendedprice AS Decimal128(38, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(38, 4))CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(38, 4))Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2))CAST(lineitem.l_discount AS Decimal128(23, 2))lineitem.l_discountDecimal128(Some(100),23,2)CAST(lineitem.l_extendedprice AS Decimal128(38, 4))lineitem.l_extendedprice AS lineitem.l_extendedprice * Decimal128(Some(100),23,2) - lineitem.l_discount) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] Projection: CAST(lineitem.l_extendedprice AS Decimal128(38, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(38, 4)) AS CAST(lineitem.l_extendedprice AS Decimal128(38, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(38, 4))CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(38, 4))Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2))CAST(lineitem.l_discount AS Decimal128(23, 2))lineitem.l_discountDecimal128(Some(100),23,2)CAST(lineitem.l_extendedprice AS Decimal128(38, 4))lineitem.l_extendedprice, part.p_type Inner Join: lineitem.l_partkey = part.p_partkey - Filter: lineitem.l_shipdate >= Date32("9374") AND lineitem.l_shipdate < Date32("9404") - TableScan: lineitem projection=[l_partkey, l_extendedprice, l_discount, l_shipdate] + Projection: lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount + Filter: lineitem.l_shipdate >= Date32("9374") AND lineitem.l_shipdate < Date32("9404") + TableScan: lineitem projection=[l_partkey, l_extendedprice, l_discount, l_shipdate] TableScan: part projection=[p_partkey, p_type] \ No newline at end of file diff --git a/benchmarks/expected-plans/q15.txt b/benchmarks/expected-plans/q15.txt index 99ec50eb84b0..0fcba46c1741 100644 --- a/benchmarks/expected-plans/q15.txt +++ b/benchmarks/expected-plans/q15.txt @@ -2,19 +2,22 @@ EmptyRelation Sort: supplier.s_suppkey ASC NULLS LAST Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue0.total_revenue Inner Join: revenue0.total_revenue = __scalar_sq_1.__value - Inner Join: supplier.s_suppkey = revenue0.supplier_no - TableScan: supplier projection=[s_suppkey, s_name, s_address, s_phone] - SubqueryAlias: revenue0 - Projection: lineitem.l_suppkey AS supplier_no, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS total_revenue - Aggregate: groupBy=[[lineitem.l_suppkey]], aggr=[[SUM(CAST(lineitem.l_extendedprice AS Decimal128(38, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(38, 4))) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] - Filter: lineitem.l_shipdate >= Date32("9496") AND lineitem.l_shipdate < Date32("9587") - TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate] + Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue0.total_revenue + Inner Join: supplier.s_suppkey = revenue0.supplier_no + TableScan: supplier projection=[s_suppkey, s_name, s_address, s_phone] + SubqueryAlias: revenue0 + Projection: lineitem.l_suppkey AS supplier_no, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS total_revenue + Aggregate: groupBy=[[lineitem.l_suppkey]], aggr=[[SUM(CAST(lineitem.l_extendedprice AS Decimal128(38, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(38, 4))) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] + Projection: lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount + Filter: lineitem.l_shipdate >= Date32("9496") AND lineitem.l_shipdate < Date32("9587") + TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate] SubqueryAlias: __scalar_sq_1 Projection: MAX(revenue0.total_revenue) AS __value Aggregate: groupBy=[[]], aggr=[[MAX(revenue0.total_revenue)]] SubqueryAlias: revenue0 Projection: SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS total_revenue Aggregate: groupBy=[[lineitem.l_suppkey]], aggr=[[SUM(CAST(lineitem.l_extendedprice AS Decimal128(38, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(38, 4))) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] - Filter: lineitem.l_shipdate >= Date32("9496") AND lineitem.l_shipdate < Date32("9587") - TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate] + Projection: lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount + Filter: lineitem.l_shipdate >= Date32("9496") AND lineitem.l_shipdate < Date32("9587") + TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate] EmptyRelation \ No newline at end of file diff --git a/benchmarks/expected-plans/q16.txt b/benchmarks/expected-plans/q16.txt index 014b1b7b4161..07bd1265c59e 100644 --- a/benchmarks/expected-plans/q16.txt +++ b/benchmarks/expected-plans/q16.txt @@ -3,10 +3,11 @@ Sort: supplier_cnt DESC NULLS FIRST, part.p_brand ASC NULLS LAST, part.p_type AS Aggregate: groupBy=[[group_alias_0, group_alias_1, group_alias_2]], aggr=[[COUNT(alias1)]] Aggregate: groupBy=[[part.p_brand AS group_alias_0, part.p_type AS group_alias_1, part.p_size AS group_alias_2, partsupp.ps_suppkey AS alias1]], aggr=[[]] LeftAnti Join: partsupp.ps_suppkey = __correlated_sq_1.s_suppkey - Inner Join: partsupp.ps_partkey = part.p_partkey - TableScan: partsupp projection=[ps_partkey, ps_suppkey] - Filter: part.p_brand != Utf8("Brand#45") AND part.p_type NOT LIKE Utf8("MEDIUM POLISHED%") AND part.p_size IN ([Int32(49), Int32(14), Int32(23), Int32(45), Int32(19), Int32(3), Int32(36), Int32(9)]) - TableScan: part projection=[p_partkey, p_brand, p_type, p_size] + Projection: partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size + Inner Join: partsupp.ps_partkey = part.p_partkey + TableScan: partsupp projection=[ps_partkey, ps_suppkey] + Filter: part.p_brand != Utf8("Brand#45") AND part.p_type NOT LIKE Utf8("MEDIUM POLISHED%") AND part.p_size IN ([Int32(49), Int32(14), Int32(23), Int32(45), Int32(19), Int32(3), Int32(36), Int32(9)]) + TableScan: part projection=[p_partkey, p_brand, p_type, p_size] SubqueryAlias: __correlated_sq_1 Projection: supplier.s_suppkey AS s_suppkey Filter: supplier.s_comment LIKE Utf8("%Customer%Complaints%") diff --git a/benchmarks/expected-plans/q17.txt b/benchmarks/expected-plans/q17.txt index 755311c5ee10..94e8aeeafe00 100644 --- a/benchmarks/expected-plans/q17.txt +++ b/benchmarks/expected-plans/q17.txt @@ -1,12 +1,16 @@ Projection: CAST(SUM(lineitem.l_extendedprice) AS Float64) / Float64(7) AS avg_yearly Aggregate: groupBy=[[]], aggr=[[SUM(lineitem.l_extendedprice)]] - Filter: CAST(lineitem.l_quantity AS Decimal128(30, 15)) < CAST(__scalar_sq_1.__value AS Decimal128(30, 15)) - Inner Join: part.p_partkey = __scalar_sq_1.l_partkey, lineitem.l_partkey = __scalar_sq_1.l_partkey - Inner Join: lineitem.l_partkey = part.p_partkey - TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice] - Filter: part.p_brand = Utf8("Brand#23") AND part.p_container = Utf8("MED BOX") - TableScan: part projection=[p_partkey, p_brand, p_container] - SubqueryAlias: __scalar_sq_1 - Projection: lineitem.l_partkey, Float64(0.2) * CAST(AVG(lineitem.l_quantity) AS Float64) AS __value - Aggregate: groupBy=[[lineitem.l_partkey]], aggr=[[AVG(lineitem.l_quantity)]] - TableScan: lineitem projection=[l_partkey, l_quantity] \ No newline at end of file + Projection: lineitem.l_extendedprice + Filter: CAST(lineitem.l_quantity AS Decimal128(30, 15)) < CAST(__scalar_sq_1.__value AS Decimal128(30, 15)) AND __scalar_sq_1.l_partkey = lineitem.l_partkey + Projection: lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, __scalar_sq_1.l_partkey, __scalar_sq_1.__value + Inner Join: part.p_partkey = __scalar_sq_1.l_partkey + Filter: part.p_partkey = lineitem.l_partkey AND lineitem.l_partkey = part.p_partkey + Inner Join: lineitem.l_partkey = part.p_partkey + TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice] + Projection: part.p_partkey + Filter: part.p_brand = Utf8("Brand#23") AND part.p_container = Utf8("MED BOX") + TableScan: part projection=[p_partkey, p_brand, p_container] + SubqueryAlias: __scalar_sq_1 + Projection: lineitem.l_partkey, Float64(0.2) * CAST(AVG(lineitem.l_quantity) AS Float64) AS __value + Aggregate: groupBy=[[lineitem.l_partkey]], aggr=[[AVG(lineitem.l_quantity)]] + TableScan: lineitem projection=[l_partkey, l_quantity] \ No newline at end of file diff --git a/benchmarks/expected-plans/q18.txt b/benchmarks/expected-plans/q18.txt index 01f60ba55f31..99e2eca1c17b 100644 --- a/benchmarks/expected-plans/q18.txt +++ b/benchmarks/expected-plans/q18.txt @@ -1,11 +1,13 @@ Sort: orders.o_totalprice DESC NULLS FIRST, orders.o_orderdate ASC NULLS LAST 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] + Projection: customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity + Inner Join: orders.o_orderkey = lineitem.l_orderkey + Projection: customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate + 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) diff --git a/benchmarks/expected-plans/q19.txt b/benchmarks/expected-plans/q19.txt index 969ad02d4c59..fd0156f422d0 100644 --- a/benchmarks/expected-plans/q19.txt +++ b/benchmarks/expected-plans/q19.txt @@ -1,8 +1,11 @@ Projection: SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue Aggregate: groupBy=[[]], aggr=[[SUM(CAST(lineitem.l_extendedprice AS Decimal128(38, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(38, 4))) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] - Filter: part.p_brand = Utf8("Brand#12") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2) AND part.p_size <= Int32(15) - Inner Join: lineitem.l_partkey = part.p_partkey - Filter: (lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2)) AND (lineitem.l_shipmode = Utf8("AIR REG") OR lineitem.l_shipmode = Utf8("AIR")) AND lineitem.l_shipinstruct = Utf8("DELIVER IN PERSON") - TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode] - Filter: (part.p_brand = Utf8("Brand#12") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND part.p_size <= Int32(15)) AND part.p_size >= Int32(1) - TableScan: part projection=[p_partkey, p_brand, p_size, p_container] + Projection: lineitem.l_extendedprice, lineitem.l_discount + Filter: part.p_brand = Utf8("Brand#12") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2) AND part.p_size <= Int32(15) + Projection: lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, part.p_brand, part.p_size, part.p_container + Inner Join: lineitem.l_partkey = part.p_partkey + Projection: lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount + Filter: (lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2)) AND (lineitem.l_shipmode = Utf8("AIR REG") OR lineitem.l_shipmode = Utf8("AIR")) AND lineitem.l_shipinstruct = Utf8("DELIVER IN PERSON") + TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode] + Filter: (part.p_brand = Utf8("Brand#12") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND part.p_size <= Int32(15)) AND part.p_size >= Int32(1) + TableScan: part projection=[p_partkey, p_brand, p_size, p_container] diff --git a/benchmarks/expected-plans/q2.txt b/benchmarks/expected-plans/q2.txt index d291548c2db8..dd1de86fcb1f 100644 --- a/benchmarks/expected-plans/q2.txt +++ b/benchmarks/expected-plans/q2.txt @@ -1,25 +1,35 @@ Sort: supplier.s_acctbal DESC NULLS FIRST, nation.n_name ASC NULLS LAST, supplier.s_name ASC NULLS LAST, part.p_partkey ASC NULLS LAST Projection: supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment Inner Join: part.p_partkey = __scalar_sq_1.ps_partkey, partsupp.ps_supplycost = __scalar_sq_1.__value - Inner Join: nation.n_regionkey = region.r_regionkey - Inner Join: supplier.s_nationkey = nation.n_nationkey - Inner Join: partsupp.ps_suppkey = supplier.s_suppkey - Inner Join: part.p_partkey = partsupp.ps_partkey - Filter: part.p_size = Int32(15) AND part.p_type LIKE Utf8("%BRASS") - TableScan: part projection=[p_partkey, p_mfgr, p_type, p_size] - TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] - TableScan: supplier projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] - TableScan: nation projection=[n_nationkey, n_name, n_regionkey] - Filter: region.r_name = Utf8("EUROPE") - TableScan: region projection=[r_regionkey, r_name] + Projection: part.p_partkey, part.p_mfgr, partsupp.ps_supplycost, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name + Inner Join: nation.n_regionkey = region.r_regionkey + Projection: part.p_partkey, part.p_mfgr, partsupp.ps_supplycost, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey + Inner Join: supplier.s_nationkey = nation.n_nationkey + Projection: part.p_partkey, part.p_mfgr, partsupp.ps_supplycost, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment + Inner Join: partsupp.ps_suppkey = supplier.s_suppkey + Projection: part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, partsupp.ps_supplycost + Inner Join: part.p_partkey = partsupp.ps_partkey + Projection: part.p_partkey, part.p_mfgr + Filter: part.p_size = Int32(15) AND part.p_type LIKE Utf8("%BRASS") + TableScan: part projection=[p_partkey, p_mfgr, p_type, p_size] + TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] + TableScan: supplier projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] + TableScan: nation projection=[n_nationkey, n_name, n_regionkey] + Projection: region.r_regionkey + Filter: region.r_name = Utf8("EUROPE") + TableScan: region projection=[r_regionkey, r_name] SubqueryAlias: __scalar_sq_1 Projection: partsupp.ps_partkey, MIN(partsupp.ps_supplycost) AS __value Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[MIN(partsupp.ps_supplycost)]] - Inner Join: nation.n_regionkey = region.r_regionkey - Inner Join: supplier.s_nationkey = nation.n_nationkey - Inner Join: partsupp.ps_suppkey = supplier.s_suppkey - TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] - TableScan: supplier projection=[s_suppkey, s_nationkey] - TableScan: nation projection=[n_nationkey, n_regionkey] - Filter: region.r_name = Utf8("EUROPE") - TableScan: region projection=[r_regionkey, r_name] \ No newline at end of file + Projection: partsupp.ps_partkey, partsupp.ps_supplycost + Inner Join: nation.n_regionkey = region.r_regionkey + Projection: partsupp.ps_partkey, partsupp.ps_supplycost, nation.n_regionkey + Inner Join: supplier.s_nationkey = nation.n_nationkey + Projection: partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey + Inner Join: partsupp.ps_suppkey = supplier.s_suppkey + TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] + TableScan: supplier projection=[s_suppkey, s_nationkey] + TableScan: nation projection=[n_nationkey, n_regionkey] + Projection: region.r_regionkey + Filter: region.r_name = Utf8("EUROPE") + TableScan: region projection=[r_regionkey, r_name] \ No newline at end of file diff --git a/benchmarks/expected-plans/q20.txt b/benchmarks/expected-plans/q20.txt index b7ecb9a09199..592c9d11de1b 100644 --- a/benchmarks/expected-plans/q20.txt +++ b/benchmarks/expected-plans/q20.txt @@ -1,22 +1,26 @@ Sort: supplier.s_name ASC NULLS LAST Projection: supplier.s_name, supplier.s_address LeftSemi Join: supplier.s_suppkey = __correlated_sq_1.ps_suppkey - Inner Join: supplier.s_nationkey = nation.n_nationkey - TableScan: supplier projection=[s_suppkey, s_name, s_address, s_nationkey] - Filter: nation.n_name = Utf8("CANADA") - TableScan: nation projection=[n_nationkey, n_name] + Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address + Inner Join: supplier.s_nationkey = nation.n_nationkey + TableScan: supplier projection=[s_suppkey, s_name, s_address, s_nationkey] + Projection: nation.n_nationkey + Filter: nation.n_name = Utf8("CANADA") + TableScan: nation projection=[n_nationkey, n_name] SubqueryAlias: __correlated_sq_1 Projection: partsupp.ps_suppkey AS ps_suppkey Filter: CAST(partsupp.ps_availqty AS Float64) > __scalar_sq_1.__value - Inner Join: partsupp.ps_partkey = __scalar_sq_1.l_partkey, partsupp.ps_suppkey = __scalar_sq_1.l_suppkey - LeftSemi Join: partsupp.ps_partkey = __correlated_sq_2.p_partkey - TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_availqty] - SubqueryAlias: __correlated_sq_2 - Projection: part.p_partkey AS p_partkey - Filter: part.p_name LIKE Utf8("forest%") - TableScan: part projection=[p_partkey, p_name] - SubqueryAlias: __scalar_sq_1 - Projection: lineitem.l_partkey, lineitem.l_suppkey, Float64(0.5) * CAST(SUM(lineitem.l_quantity) AS Float64) AS __value - Aggregate: groupBy=[[lineitem.l_partkey, lineitem.l_suppkey]], aggr=[[SUM(lineitem.l_quantity)]] - Filter: lineitem.l_shipdate >= Date32("8766") AND lineitem.l_shipdate < Date32("9131") - TableScan: lineitem projection=[l_partkey, l_suppkey, l_quantity, l_shipdate] \ No newline at end of file + Projection: partsupp.ps_suppkey, partsupp.ps_availqty, __scalar_sq_1.__value + Inner Join: partsupp.ps_partkey = __scalar_sq_1.l_partkey, partsupp.ps_suppkey = __scalar_sq_1.l_suppkey + LeftSemi Join: partsupp.ps_partkey = __correlated_sq_2.p_partkey + TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_availqty] + SubqueryAlias: __correlated_sq_2 + Projection: part.p_partkey AS p_partkey + Filter: part.p_name LIKE Utf8("forest%") + TableScan: part projection=[p_partkey, p_name] + SubqueryAlias: __scalar_sq_1 + Projection: lineitem.l_partkey, lineitem.l_suppkey, Float64(0.5) * CAST(SUM(lineitem.l_quantity) AS Float64) AS __value + Aggregate: groupBy=[[lineitem.l_partkey, lineitem.l_suppkey]], aggr=[[SUM(lineitem.l_quantity)]] + Projection: lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity + Filter: lineitem.l_shipdate >= Date32("8766") AND lineitem.l_shipdate < Date32("9131") + TableScan: lineitem projection=[l_partkey, l_suppkey, l_quantity, l_shipdate] \ No newline at end of file diff --git a/benchmarks/expected-plans/q21.txt b/benchmarks/expected-plans/q21.txt index c1d7417d8318..fcce9c2c3a6f 100644 --- a/benchmarks/expected-plans/q21.txt +++ b/benchmarks/expected-plans/q21.txt @@ -1,22 +1,29 @@ Sort: numwait DESC NULLS FIRST, supplier.s_name ASC NULLS LAST Projection: supplier.s_name, COUNT(UInt8(1)) AS numwait Aggregate: groupBy=[[supplier.s_name]], aggr=[[COUNT(UInt8(1))]] - LeftAnti Join: l1.l_orderkey = l3.l_orderkey Filter: l3.l_suppkey != l1.l_suppkey - LeftSemi Join: l1.l_orderkey = l2.l_orderkey Filter: l2.l_suppkey != l1.l_suppkey - Inner Join: supplier.s_nationkey = nation.n_nationkey - Inner Join: l1.l_orderkey = orders.o_orderkey - Inner Join: supplier.s_suppkey = l1.l_suppkey - TableScan: supplier projection=[s_suppkey, s_name, s_nationkey] - SubqueryAlias: l1 - Filter: lineitem.l_receiptdate > lineitem.l_commitdate - TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate] - Filter: orders.o_orderstatus = Utf8("F") - TableScan: orders projection=[o_orderkey, o_orderstatus] - Filter: nation.n_name = Utf8("SAUDI ARABIA") - TableScan: nation projection=[n_nationkey, n_name] - SubqueryAlias: l2 - TableScan: lineitem projection=[l_orderkey, l_suppkey] - Projection: l3.l_orderkey, l3.l_suppkey + Projection: supplier.s_name + LeftAnti Join: l1.l_orderkey = l3.l_orderkey Filter: l3.l_suppkey != l1.l_suppkey + LeftSemi Join: l1.l_orderkey = l2.l_orderkey Filter: l2.l_suppkey != l1.l_suppkey + Projection: supplier.s_name, l1.l_orderkey, l1.l_suppkey + Inner Join: supplier.s_nationkey = nation.n_nationkey + Projection: supplier.s_name, supplier.s_nationkey, l1.l_orderkey, l1.l_suppkey + Inner Join: l1.l_orderkey = orders.o_orderkey + Projection: supplier.s_name, supplier.s_nationkey, l1.l_orderkey, l1.l_suppkey + Inner Join: supplier.s_suppkey = l1.l_suppkey + TableScan: supplier projection=[s_suppkey, s_name, s_nationkey] + SubqueryAlias: l1 + Projection: lineitem.l_orderkey, lineitem.l_suppkey + Filter: lineitem.l_receiptdate > lineitem.l_commitdate + TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate] + Projection: orders.o_orderkey + Filter: orders.o_orderstatus = Utf8("F") + TableScan: orders projection=[o_orderkey, o_orderstatus] + Projection: nation.n_nationkey + Filter: nation.n_name = Utf8("SAUDI ARABIA") + TableScan: nation projection=[n_nationkey, n_name] + SubqueryAlias: l2 + TableScan: lineitem projection=[l_orderkey, l_suppkey] SubqueryAlias: l3 - Filter: lineitem.l_receiptdate > lineitem.l_commitdate - TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate] \ No newline at end of file + Projection: lineitem.l_orderkey, lineitem.l_suppkey + Filter: lineitem.l_receiptdate > lineitem.l_commitdate + TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate] \ No newline at end of file diff --git a/benchmarks/expected-plans/q22.txt b/benchmarks/expected-plans/q22.txt index 0fd7a590ac19..4b95d49e5be7 100644 --- a/benchmarks/expected-plans/q22.txt +++ b/benchmarks/expected-plans/q22.txt @@ -5,12 +5,14 @@ Sort: custsale.cntrycode ASC NULLS LAST Projection: substr(customer.c_phone, Int64(1), Int64(2)) AS cntrycode, customer.c_acctbal Filter: CAST(customer.c_acctbal AS Decimal128(19, 6)) > __scalar_sq_1.__value CrossJoin: - 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] - TableScan: orders projection=[o_custkey] + Projection: customer.c_phone, customer.c_acctbal + 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] + TableScan: orders projection=[o_custkey] SubqueryAlias: __scalar_sq_1 Projection: AVG(customer.c_acctbal) AS __value Aggregate: groupBy=[[]], aggr=[[AVG(customer.c_acctbal)]] - Filter: customer.c_acctbal > Decimal128(Some(0),15,2) AND 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_phone, c_acctbal] \ No newline at end of file + Projection: customer.c_acctbal + Filter: customer.c_acctbal > Decimal128(Some(0),15,2) AND 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_phone, c_acctbal] \ No newline at end of file diff --git a/benchmarks/expected-plans/q3.txt b/benchmarks/expected-plans/q3.txt index 7cd69b92a556..701c0aaca57d 100644 --- a/benchmarks/expected-plans/q3.txt +++ b/benchmarks/expected-plans/q3.txt @@ -1,11 +1,15 @@ Sort: revenue DESC NULLS FIRST, orders.o_orderdate ASC NULLS LAST Projection: lineitem.l_orderkey, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue, orders.o_orderdate, orders.o_shippriority Aggregate: groupBy=[[lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority]], aggr=[[SUM(CAST(lineitem.l_extendedprice AS Decimal128(38, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(38, 4))) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] - Inner Join: orders.o_orderkey = lineitem.l_orderkey - Inner Join: customer.c_custkey = orders.o_custkey - Filter: customer.c_mktsegment = Utf8("BUILDING") - TableScan: customer projection=[c_custkey, c_mktsegment] - Filter: orders.o_orderdate < Date32("9204") - TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority] - Filter: lineitem.l_shipdate > Date32("9204") - TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate] \ No newline at end of file + Projection: orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount + Inner Join: orders.o_orderkey = lineitem.l_orderkey + Projection: orders.o_orderkey, orders.o_orderdate, orders.o_shippriority + Inner Join: customer.c_custkey = orders.o_custkey + Projection: customer.c_custkey + Filter: customer.c_mktsegment = Utf8("BUILDING") + TableScan: customer projection=[c_custkey, c_mktsegment] + Filter: orders.o_orderdate < Date32("9204") + TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority] + Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount + Filter: lineitem.l_shipdate > Date32("9204") + TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate] \ No newline at end of file diff --git a/benchmarks/expected-plans/q4.txt b/benchmarks/expected-plans/q4.txt index e677f3a988a6..fec70e7412dc 100644 --- a/benchmarks/expected-plans/q4.txt +++ b/benchmarks/expected-plans/q4.txt @@ -1,9 +1,11 @@ Sort: orders.o_orderpriority ASC NULLS LAST Projection: orders.o_orderpriority, COUNT(UInt8(1)) AS order_count Aggregate: groupBy=[[orders.o_orderpriority]], aggr=[[COUNT(UInt8(1))]] - LeftSemi Join: orders.o_orderkey = lineitem.l_orderkey - Filter: orders.o_orderdate >= Date32("8582") AND orders.o_orderdate < Date32("8674") - TableScan: orders projection=[o_orderkey, o_orderdate, o_orderpriority] - Projection: lineitem.l_orderkey - Filter: lineitem.l_commitdate < lineitem.l_receiptdate - TableScan: lineitem projection=[l_orderkey, l_commitdate, l_receiptdate] \ No newline at end of file + Projection: orders.o_orderpriority + LeftSemi Join: orders.o_orderkey = lineitem.l_orderkey + Projection: orders.o_orderkey, orders.o_orderpriority + Filter: orders.o_orderdate >= Date32("8582") AND orders.o_orderdate < Date32("8674") + TableScan: orders projection=[o_orderkey, o_orderdate, o_orderpriority] + Projection: lineitem.l_orderkey + Filter: lineitem.l_commitdate < lineitem.l_receiptdate + TableScan: lineitem projection=[l_orderkey, l_commitdate, l_receiptdate] \ No newline at end of file diff --git a/benchmarks/expected-plans/q5.txt b/benchmarks/expected-plans/q5.txt index 0c2e0c131805..e220308c5739 100644 --- a/benchmarks/expected-plans/q5.txt +++ b/benchmarks/expected-plans/q5.txt @@ -1,16 +1,23 @@ Sort: revenue DESC NULLS FIRST Projection: nation.n_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue Aggregate: groupBy=[[nation.n_name]], aggr=[[SUM(CAST(lineitem.l_extendedprice AS Decimal128(38, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(38, 4))) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] - Inner Join: nation.n_regionkey = region.r_regionkey - Inner Join: supplier.s_nationkey = nation.n_nationkey - Inner Join: lineitem.l_suppkey = supplier.s_suppkey, customer.c_nationkey = supplier.s_nationkey - Inner Join: orders.o_orderkey = lineitem.l_orderkey - Inner Join: customer.c_custkey = orders.o_custkey - TableScan: customer projection=[c_custkey, c_nationkey] - Filter: orders.o_orderdate >= Date32("8766") AND orders.o_orderdate < Date32("9131") - TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate] - TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount] - TableScan: supplier projection=[s_suppkey, s_nationkey] - TableScan: nation projection=[n_nationkey, n_name, n_regionkey] - Filter: region.r_name = Utf8("ASIA") - TableScan: region projection=[r_regionkey, r_name] \ No newline at end of file + Projection: lineitem.l_extendedprice, lineitem.l_discount, nation.n_name + Inner Join: nation.n_regionkey = region.r_regionkey + Projection: lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_regionkey + Inner Join: supplier.s_nationkey = nation.n_nationkey + Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey + Inner Join: lineitem.l_suppkey = supplier.s_suppkey, customer.c_nationkey = supplier.s_nationkey + Projection: customer.c_nationkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount + Inner Join: orders.o_orderkey = lineitem.l_orderkey + Projection: customer.c_nationkey, orders.o_orderkey + Inner Join: customer.c_custkey = orders.o_custkey + TableScan: customer projection=[c_custkey, c_nationkey] + Projection: orders.o_orderkey, orders.o_custkey + Filter: orders.o_orderdate >= Date32("8766") AND orders.o_orderdate < Date32("9131") + TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate] + TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount] + TableScan: supplier projection=[s_suppkey, s_nationkey] + TableScan: nation projection=[n_nationkey, n_name, n_regionkey] + Projection: region.r_regionkey + Filter: region.r_name = Utf8("ASIA") + TableScan: region projection=[r_regionkey, r_name] \ No newline at end of file diff --git a/benchmarks/expected-plans/q6.txt b/benchmarks/expected-plans/q6.txt index 463909acbc68..720b75ed82df 100644 --- a/benchmarks/expected-plans/q6.txt +++ b/benchmarks/expected-plans/q6.txt @@ -1,4 +1,5 @@ Projection: SUM(lineitem.l_extendedprice * lineitem.l_discount) AS revenue Aggregate: groupBy=[[]], aggr=[[SUM(lineitem.l_extendedprice * lineitem.l_discount)]] - Filter: lineitem.l_shipdate >= Date32("8766") AND lineitem.l_shipdate < Date32("9131") AND lineitem.l_discount >= Decimal128(Some(5),15,2) AND lineitem.l_discount <= Decimal128(Some(7),15,2) AND lineitem.l_quantity < Decimal128(Some(2400),15,2) - TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_shipdate] + Projection: lineitem.l_extendedprice, lineitem.l_discount + Filter: lineitem.l_shipdate >= Date32("8766") AND lineitem.l_shipdate < Date32("9131") AND lineitem.l_discount >= Decimal128(Some(5),15,2) AND lineitem.l_discount <= Decimal128(Some(7),15,2) AND lineitem.l_quantity < Decimal128(Some(2400),15,2) + TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_shipdate] \ No newline at end of file diff --git a/benchmarks/expected-plans/q7.txt b/benchmarks/expected-plans/q7.txt index bd8c10f8cf88..db0d56c9ddb5 100644 --- a/benchmarks/expected-plans/q7.txt +++ b/benchmarks/expected-plans/q7.txt @@ -4,19 +4,24 @@ Sort: shipping.supp_nation ASC NULLS LAST, shipping.cust_nation ASC NULLS LAST, SubqueryAlias: shipping Projection: n1.n_name AS supp_nation, n2.n_name AS cust_nation, datepart(Utf8("YEAR"), lineitem.l_shipdate) AS l_year, CAST(lineitem.l_extendedprice AS Decimal128(38, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(38, 4)) AS volume Filter: n1.n_name = Utf8("FRANCE") AND n2.n_name = Utf8("GERMANY") OR n1.n_name = Utf8("GERMANY") AND n2.n_name = Utf8("FRANCE") - Inner Join: customer.c_nationkey = n2.n_nationkey - Inner Join: supplier.s_nationkey = n1.n_nationkey - Inner Join: orders.o_custkey = customer.c_custkey - Inner Join: lineitem.l_orderkey = orders.o_orderkey - Inner Join: supplier.s_suppkey = lineitem.l_suppkey - TableScan: supplier projection=[s_suppkey, s_nationkey] - Filter: lineitem.l_shipdate >= Date32("9131") AND lineitem.l_shipdate <= Date32("9861") - TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate] - TableScan: orders projection=[o_orderkey, o_custkey] - TableScan: customer projection=[c_custkey, c_nationkey] - SubqueryAlias: n1 - Filter: nation.n_name = Utf8("FRANCE") OR nation.n_name = Utf8("GERMANY") - TableScan: nation projection=[n_nationkey, n_name] - SubqueryAlias: n2 - Filter: nation.n_name = Utf8("GERMANY") OR nation.n_name = Utf8("FRANCE") - TableScan: nation projection=[n_nationkey, n_name] \ No newline at end of file + Projection: lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, n1.n_name, n2.n_name + Inner Join: customer.c_nationkey = n2.n_nationkey + Projection: lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, customer.c_nationkey, n1.n_name + Inner Join: supplier.s_nationkey = n1.n_nationkey + Projection: supplier.s_nationkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, customer.c_nationkey + Inner Join: orders.o_custkey = customer.c_custkey + Projection: supplier.s_nationkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, orders.o_custkey + Inner Join: lineitem.l_orderkey = orders.o_orderkey + Projection: supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate + Inner Join: supplier.s_suppkey = lineitem.l_suppkey + TableScan: supplier projection=[s_suppkey, s_nationkey] + Filter: lineitem.l_shipdate >= Date32("9131") AND lineitem.l_shipdate <= Date32("9861") + TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate] + TableScan: orders projection=[o_orderkey, o_custkey] + TableScan: customer projection=[c_custkey, c_nationkey] + SubqueryAlias: n1 + Filter: nation.n_name = Utf8("FRANCE") OR nation.n_name = Utf8("GERMANY") + TableScan: nation projection=[n_nationkey, n_name] + SubqueryAlias: n2 + Filter: nation.n_name = Utf8("GERMANY") OR nation.n_name = Utf8("FRANCE") + TableScan: nation projection=[n_nationkey, n_name] \ No newline at end of file diff --git a/benchmarks/expected-plans/q8.txt b/benchmarks/expected-plans/q8.txt index e8c47b2ab7d3..0bddf8c3d680 100644 --- a/benchmarks/expected-plans/q8.txt +++ b/benchmarks/expected-plans/q8.txt @@ -4,22 +4,30 @@ Sort: all_nations.o_year ASC NULLS LAST SubqueryAlias: all_nations Projection: datepart(Utf8("YEAR"), orders.o_orderdate) AS o_year, CAST(lineitem.l_extendedprice AS Decimal128(38, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(38, 4)) AS volume, n2.n_name AS nation Inner Join: n1.n_regionkey = region.r_regionkey - Inner Join: supplier.s_nationkey = n2.n_nationkey - Inner Join: customer.c_nationkey = n1.n_nationkey - Inner Join: orders.o_custkey = customer.c_custkey - Inner Join: lineitem.l_orderkey = orders.o_orderkey - Inner Join: lineitem.l_suppkey = supplier.s_suppkey - Inner Join: part.p_partkey = lineitem.l_partkey - Filter: part.p_type = Utf8("ECONOMY ANODIZED STEEL") - TableScan: part projection=[p_partkey, p_type] - TableScan: lineitem projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount] - TableScan: supplier projection=[s_suppkey, s_nationkey] - Filter: orders.o_orderdate >= Date32("9131") AND orders.o_orderdate <= Date32("9861") - TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate] - TableScan: customer projection=[c_custkey, c_nationkey] - SubqueryAlias: n1 - TableScan: nation projection=[n_nationkey, n_regionkey] - SubqueryAlias: n2 - TableScan: nation projection=[n_nationkey, n_name] - Filter: region.r_name = Utf8("AMERICA") - TableScan: region projection=[r_regionkey, r_name] \ No newline at end of file + Projection: lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, n1.n_regionkey, n2.n_name + Inner Join: supplier.s_nationkey = n2.n_nationkey + Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_orderdate, n1.n_regionkey + Inner Join: customer.c_nationkey = n1.n_nationkey + Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_orderdate, customer.c_nationkey + Inner Join: orders.o_custkey = customer.c_custkey + Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_custkey, orders.o_orderdate + Inner Join: lineitem.l_orderkey = orders.o_orderkey + Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey + Inner Join: lineitem.l_suppkey = supplier.s_suppkey + Projection: lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount + Inner Join: part.p_partkey = lineitem.l_partkey + Projection: part.p_partkey + Filter: part.p_type = Utf8("ECONOMY ANODIZED STEEL") + TableScan: part projection=[p_partkey, p_type] + TableScan: lineitem projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount] + TableScan: supplier projection=[s_suppkey, s_nationkey] + Filter: orders.o_orderdate >= Date32("9131") AND orders.o_orderdate <= Date32("9861") + TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate] + TableScan: customer projection=[c_custkey, c_nationkey] + SubqueryAlias: n1 + TableScan: nation projection=[n_nationkey, n_regionkey] + SubqueryAlias: n2 + TableScan: nation projection=[n_nationkey, n_name] + Projection: region.r_regionkey + Filter: region.r_name = Utf8("AMERICA") + TableScan: region projection=[r_regionkey, r_name] \ No newline at end of file diff --git a/benchmarks/expected-plans/q9.txt b/benchmarks/expected-plans/q9.txt index c83f9d945da5..902cc4a12928 100644 --- a/benchmarks/expected-plans/q9.txt +++ b/benchmarks/expected-plans/q9.txt @@ -4,14 +4,19 @@ Sort: profit.nation ASC NULLS LAST, profit.o_year DESC NULLS FIRST SubqueryAlias: profit Projection: nation.n_name AS nation, datepart(Utf8("YEAR"), orders.o_orderdate) AS o_year, CAST(lineitem.l_extendedprice AS Decimal128(38, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(38, 4)) - CAST(partsupp.ps_supplycost * lineitem.l_quantity AS Decimal128(38, 4)) AS amount Inner Join: supplier.s_nationkey = nation.n_nationkey - Inner Join: lineitem.l_orderkey = orders.o_orderkey - Inner Join: lineitem.l_suppkey = partsupp.ps_suppkey, lineitem.l_partkey = partsupp.ps_partkey - Inner Join: lineitem.l_suppkey = supplier.s_suppkey - Inner Join: part.p_partkey = lineitem.l_partkey - Filter: part.p_name LIKE Utf8("%green%") - TableScan: part projection=[p_partkey, p_name] - TableScan: lineitem projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount] - TableScan: supplier projection=[s_suppkey, s_nationkey] - TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] - TableScan: orders projection=[o_orderkey, o_orderdate] + Projection: lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, partsupp.ps_supplycost, orders.o_orderdate + Inner Join: lineitem.l_orderkey = orders.o_orderkey + Projection: lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, partsupp.ps_supplycost + Inner Join: lineitem.l_suppkey = partsupp.ps_suppkey, lineitem.l_partkey = partsupp.ps_partkey + Projection: lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey + Inner Join: lineitem.l_suppkey = supplier.s_suppkey + Projection: lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount + Inner Join: part.p_partkey = lineitem.l_partkey + Projection: part.p_partkey + Filter: part.p_name LIKE Utf8("%green%") + TableScan: part projection=[p_partkey, p_name] + TableScan: lineitem projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount] + TableScan: supplier projection=[s_suppkey, s_nationkey] + TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] + TableScan: orders projection=[o_orderkey, o_orderdate] TableScan: nation projection=[n_nationkey, n_name] \ No newline at end of file diff --git a/datafusion/core/tests/dataframe.rs b/datafusion/core/tests/dataframe.rs index a9e28848cc0e..7a6e11ca258d 100644 --- a/datafusion/core/tests/dataframe.rs +++ b/datafusion/core/tests/dataframe.rs @@ -503,8 +503,9 @@ async fn right_semi_with_alias_filter() -> Result<()> { let optimized_plan = df.clone().into_optimized_plan()?; let expected = vec![ "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]", + " Projection: t1.a [a:UInt32]", + " 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]", ]; @@ -547,8 +548,9 @@ async fn right_anti_filter_push_down() -> Result<()> { let optimized_plan = df.clone().into_optimized_plan()?; let expected = vec![ "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]", + " Projection: t1.a [a:UInt32]", + " 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]", ]; diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 3cbf50275326..6f7150d2a53b 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -758,7 +758,8 @@ async fn explain_logical_plan_only() { "logical_plan", "Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1))]]\ \n SubqueryAlias: t\ - \n Values: (Utf8(\"a\"), Int64(1), Int64(100)), (Utf8(\"a\"), Int64(2), Int64(150))", + \n Projection: column1\ + \n Values: (Utf8(\"a\"), Int64(1), Int64(100)), (Utf8(\"a\"), Int64(2), Int64(150))" ]]; assert_eq!(expected, actual); } diff --git a/datafusion/core/tests/sql/joins.rs b/datafusion/core/tests/sql/joins.rs index 5675a4bd6c5f..e021cdc59c43 100644 --- a/datafusion/core/tests/sql/joins.rs +++ b/datafusion/core/tests/sql/joins.rs @@ -1600,11 +1600,12 @@ async fn reduce_left_join_3() -> Result<()> { " 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]", + " Inner Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id: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]", + " Projection: t2.t2_id [t2_id: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(); @@ -2104,8 +2105,9 @@ async fn left_semi_join_pushdown() -> Result<()> { "Explain [plan_type:Utf8, plan:Utf8]", " 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]", + " Projection: t2.t2_id [t2_id:UInt32;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(); @@ -3268,13 +3270,13 @@ async fn right_as_inner_table_nested_loop_join() -> Result<()> { // right 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=Inner, filter=BinaryExpr { left: Column { name: \"t1_id\", index: 0 }, op: Gt, right: Column { name: \"t2_id\", index: 1 } }", - " CoalesceBatchesExec: target_batch_size=4096", - " FilterExec: t1_id@0 > 10", - " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1]", - " CoalescePartitionsExec", + "NestedLoopJoinExec: join_type=Inner, filter=BinaryExpr { left: Column { name: \"t1_id\", index: 0 }, op: Gt, right: Column { name: \"t2_id\", index: 1 } }", + " CoalesceBatchesExec: target_batch_size=4096", + " FilterExec: t1_id@0 > 10", + " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1]", + " CoalescePartitionsExec", + " ProjectionExec: expr=[t2_id@0 as t2_id]", " CoalesceBatchesExec: target_batch_size=4096", " FilterExec: t2_int@1 > 1", " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", diff --git a/datafusion/core/tests/sql/predicates.rs b/datafusion/core/tests/sql/predicates.rs index ac57e0a1f224..cf8bf998e070 100644 --- a/datafusion/core/tests/sql/predicates.rs +++ b/datafusion/core/tests/sql/predicates.rs @@ -100,12 +100,13 @@ async fn multiple_or_predicates() -> Result<()> { let expected = vec![ "Explain [plan_type:Utf8, plan:Utf8]", " Projection: lineitem.l_partkey [l_partkey:Int64]", - " Filter: part.p_brand = Utf8(\"Brand#12\") AND lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) AND part.p_size <= Int32(5) OR part.p_brand = Utf8(\"Brand#23\") AND lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) AND part.p_size <= Int32(10) OR part.p_brand = Utf8(\"Brand#34\") AND lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2) AND part.p_size <= Int32(15) [l_partkey:Int64, l_quantity:Decimal128(15, 2), p_partkey:Int64, p_brand:Utf8, p_size:Int32]", - " Inner Join: lineitem.l_partkey = part.p_partkey [l_partkey:Int64, l_quantity:Decimal128(15, 2), p_partkey:Int64, p_brand:Utf8, p_size:Int32]", - " Filter: lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2) [l_partkey:Int64, l_quantity:Decimal128(15, 2)]", - " TableScan: lineitem projection=[l_partkey, l_quantity], partial_filters=[lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2)] [l_partkey:Int64, l_quantity:Decimal128(15, 2)]", - " Filter: (part.p_brand = Utf8(\"Brand#12\") AND part.p_size <= Int32(5) OR part.p_brand = Utf8(\"Brand#23\") AND part.p_size <= Int32(10) OR part.p_brand = Utf8(\"Brand#34\") AND part.p_size <= Int32(15)) AND part.p_size >= Int32(1) [p_partkey:Int64, p_brand:Utf8, p_size:Int32]", - " TableScan: part projection=[p_partkey, p_brand, p_size], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8(\"Brand#12\") AND part.p_size <= Int32(5) OR part.p_brand = Utf8(\"Brand#23\") AND part.p_size <= Int32(10) OR part.p_brand = Utf8(\"Brand#34\") AND part.p_size <= Int32(15)] [p_partkey:Int64, p_brand:Utf8, p_size:Int32]", + " Filter: part.p_brand = Utf8(\"Brand#12\") AND lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) AND part.p_size <= Int32(5) OR part.p_brand = Utf8(\"Brand#23\") AND lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) AND part.p_size <= Int32(10) OR part.p_brand = Utf8(\"Brand#34\") AND lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2) AND part.p_size <= Int32(15) [l_partkey:Int64, l_quantity:Decimal128(15, 2), p_brand:Utf8, p_size:Int32]", + " Projection: lineitem.l_partkey, lineitem.l_quantity, part.p_brand, part.p_size [l_partkey:Int64, l_quantity:Decimal128(15, 2), p_brand:Utf8, p_size:Int32]", + " Inner Join: lineitem.l_partkey = part.p_partkey [l_partkey:Int64, l_quantity:Decimal128(15, 2), p_partkey:Int64, p_brand:Utf8, p_size:Int32]", + " Filter: lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2) [l_partkey:Int64, l_quantity:Decimal128(15, 2)]", + " TableScan: lineitem projection=[l_partkey, l_quantity], partial_filters=[lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2)] [l_partkey:Int64, l_quantity:Decimal128(15, 2)]", + " Filter: (part.p_brand = Utf8(\"Brand#12\") AND part.p_size <= Int32(5) OR part.p_brand = Utf8(\"Brand#23\") AND part.p_size <= Int32(10) OR part.p_brand = Utf8(\"Brand#34\") AND part.p_size <= Int32(15)) AND part.p_size >= Int32(1) [p_partkey:Int64, p_brand:Utf8, p_size:Int32]", + " TableScan: part projection=[p_partkey, p_brand, p_size], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8(\"Brand#12\") AND part.p_size <= Int32(5) OR part.p_brand = Utf8(\"Brand#23\") AND part.p_size <= Int32(10) OR part.p_brand = Utf8(\"Brand#34\") AND part.p_size <= Int32(15)] [p_partkey:Int64, p_brand:Utf8, p_size:Int32]", ]; let formatted = plan.display_indent_schema().to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -158,12 +159,15 @@ where let actual: Vec<&str> = formatted.trim().lines().collect(); 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]", + " Projection: lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey, partsupp.ps_suppkey [l_extendedprice:Decimal128(15, 2), l_discount:Decimal128(15, 2), p_partkey:Int64, ps_suppkey:Int64]", + " Inner Join: part.p_partkey = partsupp.ps_partkey [l_extendedprice:Decimal128(15, 2), l_discount:Decimal128(15, 2), p_partkey:Int64, ps_partkey:Int64, ps_suppkey:Int64]", + " Projection: lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey [l_extendedprice:Decimal128(15, 2), l_discount:Decimal128(15, 2), p_partkey: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]", + " TableScan: lineitem projection=[l_partkey, l_extendedprice, l_discount] [l_partkey:Int64, l_extendedprice:Decimal128(15, 2), l_discount:Decimal128(15, 2)]", + " Projection: part.p_partkey [p_partkey:Int64]", + " 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/subqueries.rs b/datafusion/core/tests/sql/subqueries.rs index fb114d641567..06d08d2e9f96 100644 --- a/datafusion/core/tests/sql/subqueries.rs +++ b/datafusion/core/tests/sql/subqueries.rs @@ -50,21 +50,24 @@ where c_acctbal < ( let plan = dataframe.into_optimized_plan().unwrap(); let actual = format!("{}", plan.display_indent()); - let expected = r#"Sort: customer.c_custkey ASC NULLS LAST - Projection: customer.c_custkey - Filter: CAST(customer.c_acctbal AS Decimal128(25, 2)) < __scalar_sq_1.__value - Inner Join: customer.c_custkey = __scalar_sq_1.o_custkey - TableScan: customer projection=[c_custkey, c_acctbal] - SubqueryAlias: __scalar_sq_1 - Projection: orders.o_custkey, SUM(orders.o_totalprice) AS __value - Aggregate: groupBy=[[orders.o_custkey]], aggr=[[SUM(orders.o_totalprice)]] - Filter: CAST(orders.o_totalprice AS Decimal128(25, 2)) < __scalar_sq_2.__value - Inner Join: orders.o_orderkey = __scalar_sq_2.l_orderkey - TableScan: orders projection=[o_orderkey, o_custkey, o_totalprice] - SubqueryAlias: __scalar_sq_2 - Projection: lineitem.l_orderkey, SUM(lineitem.l_extendedprice) AS price AS __value - Aggregate: groupBy=[[lineitem.l_orderkey]], aggr=[[SUM(lineitem.l_extendedprice)]] - TableScan: lineitem projection=[l_orderkey, l_extendedprice]"#; + let expected = "Sort: customer.c_custkey ASC NULLS LAST\ + \n Projection: customer.c_custkey\ + \n Filter: CAST(customer.c_acctbal AS Decimal128(25, 2)) < __scalar_sq_1.__value\ + \n Projection: customer.c_custkey, customer.c_acctbal, __scalar_sq_1.__value\ + \n Inner Join: customer.c_custkey = __scalar_sq_1.o_custkey\ + \n TableScan: customer projection=[c_custkey, c_acctbal]\ + \n SubqueryAlias: __scalar_sq_1\ + \n Projection: orders.o_custkey, SUM(orders.o_totalprice) AS __value\ + \n Aggregate: groupBy=[[orders.o_custkey]], aggr=[[SUM(orders.o_totalprice)]]\ + \n Projection: orders.o_custkey, orders.o_totalprice\ + \n Filter: CAST(orders.o_totalprice AS Decimal128(25, 2)) < __scalar_sq_2.__value\ + \n Projection: orders.o_custkey, orders.o_totalprice, __scalar_sq_2.__value\ + \n Inner Join: orders.o_orderkey = __scalar_sq_2.l_orderkey\ + \n TableScan: orders projection=[o_orderkey, o_custkey, o_totalprice]\ + \n SubqueryAlias: __scalar_sq_2\ + \n Projection: lineitem.l_orderkey, SUM(lineitem.l_extendedprice) AS price AS __value\ + \n Aggregate: groupBy=[[lineitem.l_orderkey]], aggr=[[SUM(lineitem.l_extendedprice)]]\ + \n TableScan: lineitem projection=[l_orderkey, l_extendedprice]"; assert_eq!(actual, expected); Ok(()) diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 329eaee529cf..5cd91e46ea9f 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -81,10 +81,10 @@ async fn test_window_agg_sort() -> Result<()> { // Only 1 SortExec was added let expected = { vec![ - "ProjectionExec: expr=[c9@1 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum2]", + "ProjectionExec: expr=[c9@8 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@14 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@13 as sum2]", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(NULL)), end_bound: CurrentRow }]", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(NULL)), end_bound: CurrentRow }]", - " SortExec: expr=[c9@1 ASC NULLS LAST,c8@0 ASC NULLS LAST]", + " SortExec: expr=[c9@8 ASC NULLS LAST,c8@7 ASC NULLS LAST]", ] }; @@ -112,11 +112,11 @@ async fn over_order_by_sort_keys_sorting_prefix_compacting() -> Result<()> { // Only 1 SortExec was added let expected = { vec![ - "ProjectionExec: expr=[c2@0 as c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as MAX(aggregate_test_100.c9), SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9), MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as MIN(aggregate_test_100.c9)]", + "ProjectionExec: expr=[c2@1 as c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@14 as MAX(aggregate_test_100.c9), SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@15 as SUM(aggregate_test_100.c9), MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@13 as MIN(aggregate_test_100.c9)]", " WindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }]", " BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c9): Ok(Field { name: \"MAX(aggregate_test_100.c9)\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(NULL)), end_bound: CurrentRow }]", " BoundedWindowAggExec: wdw=[MIN(aggregate_test_100.c9): Ok(Field { name: \"MIN(aggregate_test_100.c9)\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(NULL)), end_bound: CurrentRow }]", - " SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST]", + " SortExec: expr=[c2@1 ASC NULLS LAST,c9@8 ASC NULLS LAST]", ] }; @@ -145,12 +145,12 @@ async fn over_order_by_sort_keys_sorting_global_order_compacting() -> Result<()> let expected = { vec![ "SortExec: expr=[c2@0 ASC NULLS LAST]", - " ProjectionExec: expr=[c2@0 as c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as MAX(aggregate_test_100.c9), SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9), MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as MIN(aggregate_test_100.c9)]", + " ProjectionExec: expr=[c2@1 as c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@14 as MAX(aggregate_test_100.c9), SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@15 as SUM(aggregate_test_100.c9), MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@13 as MIN(aggregate_test_100.c9)]", " WindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }]", " BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c9): Ok(Field { name: \"MAX(aggregate_test_100.c9)\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(NULL)), end_bound: CurrentRow }]", - " SortExec: expr=[c9@1 ASC NULLS LAST,c2@0 ASC NULLS LAST]", + " SortExec: expr=[c9@8 ASC NULLS LAST,c2@1 ASC NULLS LAST]", " BoundedWindowAggExec: wdw=[MIN(aggregate_test_100.c9): Ok(Field { name: \"MIN(aggregate_test_100.c9)\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(NULL)), end_bound: CurrentRow }]", - " SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST]", + " SortExec: expr=[c2@1 ASC NULLS LAST,c9@8 ASC NULLS LAST]", ] }; @@ -184,7 +184,7 @@ async fn test_window_partition_by_order_by() -> Result<()> { let formatted = displayable(physical_plan.as_ref()).indent().to_string(); let expected = { vec![ - "ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as SUM(aggregate_test_100.c4), COUNT(UInt8(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as COUNT(UInt8(1))]", + "ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@13 as SUM(aggregate_test_100.c4), COUNT(UInt8(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@14 as COUNT(UInt8(1))]", " BoundedWindowAggExec: wdw=[COUNT(UInt8(1)): Ok(Field { name: \"COUNT(UInt8(1))\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }]", " SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=4096", @@ -225,11 +225,11 @@ async fn test_window_agg_sort_reversed_plan() -> Result<()> { // Only 1 SortExec was added let expected = { vec![ - "ProjectionExec: expr=[c9@0 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as sum2]", + "ProjectionExec: expr=[c9@8 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@14 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@13 as sum2]", " GlobalLimitExec: skip=0, fetch=5", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)) }]", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " SortExec: expr=[c9@0 DESC]", + " SortExec: expr=[c9@8 DESC]", ] }; @@ -280,11 +280,11 @@ async fn test_window_agg_sort_reversed_plan_builtin() -> Result<()> { // Only 1 SortExec was added let expected = { vec![ - "ProjectionExec: expr=[c9@0 as c9, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as fv1, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as fv2, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as lag1, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as lag2, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as lead1, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as lead2]", + "ProjectionExec: expr=[c9@8 as c9, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@16 as fv1, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@13 as fv2, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@17 as lag1, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as lag2, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@18 as lead1, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as lead2]", " GlobalLimitExec: skip=0, fetch=5", " BoundedWindowAggExec: wdw=[FIRST_VALUE(aggregate_test_100.c9): Ok(Field { name: \"FIRST_VALUE(aggregate_test_100.c9)\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)) }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)): Ok(Field { name: \"LAG(aggregate_test_100.c9,Int64(2),Int64(10101))\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(UInt32(NULL)) }, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)): Ok(Field { name: \"LEAD(aggregate_test_100.c9,Int64(2),Int64(10101))\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(UInt32(NULL)) }]", " BoundedWindowAggExec: wdw=[FIRST_VALUE(aggregate_test_100.c9): Ok(Field { name: \"FIRST_VALUE(aggregate_test_100.c9)\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)): Ok(Field { name: \"LAG(aggregate_test_100.c9,Int64(2),Int64(10101))\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)): Ok(Field { name: \"LEAD(aggregate_test_100.c9,Int64(2),Int64(10101))\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }]", - " SortExec: expr=[c9@0 DESC]", + " SortExec: expr=[c9@8 DESC]", ] }; @@ -331,12 +331,12 @@ async fn test_window_agg_sort_non_reversed_plan() -> Result<()> { // We cannot reverse each window function (ROW_NUMBER is not reversible) let expected = { vec![ - "ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as rn1, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as rn2]", + "ProjectionExec: expr=[c9@8 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@14 as rn1, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@13 as rn2]", " GlobalLimitExec: skip=0, fetch=5", " BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " SortExec: expr=[c9@0 ASC NULLS LAST]", + " SortExec: expr=[c9@8 ASC NULLS LAST]", " BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " SortExec: expr=[c9@0 DESC]" + " SortExec: expr=[c9@8 DESC]", ] }; @@ -384,13 +384,13 @@ async fn test_window_agg_sort_multi_layer_non_reversed_plan() -> Result<()> { // We cannot reverse each window function (ROW_NUMBER is not reversible) let expected = { vec![ - "ProjectionExec: expr=[c9@2 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as rn2]", + "ProjectionExec: expr=[c9@8 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@15 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@13 as sum2, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@14 as rn2]", " GlobalLimitExec: skip=0, fetch=5", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " SortExec: expr=[c9@2 ASC NULLS LAST,c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST]", + " SortExec: expr=[c9@8 ASC NULLS LAST,c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST]", " BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " SortExec: expr=[c9@2 DESC,c1@0 DESC]", + " SortExec: expr=[c9@8 DESC,c1@0 DESC]", ] }; @@ -519,11 +519,11 @@ async fn test_window_agg_sort_orderby_reversed_partitionby_plan() -> Result<()> // Only 1 SortExec was added let expected = { vec![ - "ProjectionExec: expr=[c9@1 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2]", + "ProjectionExec: expr=[c9@8 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@13 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@14 as sum2]", " GlobalLimitExec: skip=0, fetch=5", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC]", + " SortExec: expr=[c1@0 ASC NULLS LAST,c9@8 DESC]", ] }; @@ -573,11 +573,11 @@ async fn test_window_agg_sort_partitionby_reversed_plan() -> Result<()> { // Only 1 SortExec was added let expected = { vec![ - "ProjectionExec: expr=[c9@1 as c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as sum2]", + "ProjectionExec: expr=[c9@8 as c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@14 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@13 as sum2]", " GlobalLimitExec: skip=0, fetch=5", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)) }]", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC]", + " SortExec: expr=[c1@0 ASC NULLS LAST,c9@8 DESC]", ] }; @@ -626,11 +626,11 @@ async fn test_window_agg_sort_orderby_reversed_binary_expr() -> Result<()> { // Only 1 SortExec was added let expected = { vec![ - "ProjectionExec: expr=[c3@1 as c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum2]", + "ProjectionExec: expr=[c3@2 as c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@13 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@14 as sum2]", " GlobalLimitExec: skip=0, fetch=5", " WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: \"SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int16(NULL)) }]", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: \"SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow }]", - " SortExec: expr=[CAST(c3@1 AS Int16) + c4@2 DESC,c9@3 DESC,c2@0 ASC NULLS LAST]", + " SortExec: expr=[CAST(c3@2 AS Int16) + c4@3 DESC,c9@8 DESC,c2@1 ASC NULLS LAST]", ] }; @@ -686,13 +686,15 @@ async fn test_remove_unnecessary_sort_in_sub_query() -> Result<()> { " CoalescePartitionsExec", " AggregateExec: mode=Partial, gby=[], aggr=[COUNT(UInt8(1))]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=8", - " AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[COUNT(UInt8(1))]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 8), input_partitions=8", - " AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[COUNT(UInt8(1))]", - " CoalesceBatchesExec: target_batch_size=4096", - " FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " ProjectionExec: expr=[c1@0 as c1]", + " AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[COUNT(UInt8(1))]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 8), input_partitions=8", + " AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[COUNT(UInt8(1))]", + " ProjectionExec: expr=[c1@0 as c1]", + " CoalesceBatchesExec: target_batch_size=4096", + " FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ] }; @@ -737,11 +739,11 @@ async fn test_window_agg_sort_orderby_reversed_partitionby_reversed_plan() -> Re // Only 1 SortExec was added let expected = { vec![ - "ProjectionExec: expr=[c3@1 as c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum2]", + "ProjectionExec: expr=[c3@2 as c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@13 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@14 as sum2]", " GlobalLimitExec: skip=0, fetch=5", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(NULL)), end_bound: CurrentRow }]", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow }]", - " SortExec: expr=[c3@1 DESC,c9@2 DESC,c2@0 ASC NULLS LAST]", + " SortExec: expr=[c3@2 DESC,c9@8 DESC,c2@1 ASC NULLS LAST]", ] }; @@ -788,7 +790,7 @@ async fn test_window_agg_global_sort() -> Result<()> { let expected = { vec![ "SortPreservingMergeExec: [c1@0 ASC NULLS LAST]", - " ProjectionExec: expr=[c1@0 as c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1]", + " ProjectionExec: expr=[c1@0 as c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@13 as rn1]", " BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }]", " SortExec: expr=[c1@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", @@ -827,7 +829,7 @@ async fn test_window_agg_global_sort_parallelize_sort_disabled() -> Result<()> { vec![ "SortExec: expr=[c1@0 ASC NULLS LAST]", " CoalescePartitionsExec", - " ProjectionExec: expr=[c1@0 as c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1]", + " ProjectionExec: expr=[c1@0 as c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@13 as rn1]", " BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }]", " SortExec: expr=[c1@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", @@ -868,12 +870,12 @@ async fn test_window_agg_global_sort_intermediate_parallel_sort() -> Result<()> let expected = { vec![ "SortExec: expr=[c1@0 ASC NULLS LAST]", - " ProjectionExec: expr=[c1@0 as c1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2]", + " ProjectionExec: expr=[c1@0 as c1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING@13 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@14 as sum2]", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " SortPreservingMergeExec: [c9@1 ASC NULLS LAST]", - " SortExec: expr=[c9@1 ASC NULLS LAST]", + " SortPreservingMergeExec: [c9@8 ASC NULLS LAST]", + " SortExec: expr=[c9@8 ASC NULLS LAST]", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(3)) }]", - " SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 ASC NULLS LAST]", + " SortExec: expr=[c1@0 ASC NULLS LAST,c9@8 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 2), input_partitions=2", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", @@ -1180,7 +1182,7 @@ mod tests { "ProjectionExec: expr=[fv1@0 as fv1, fv2@1 as fv2, lv1@2 as lv1, lv2@3 as lv2, nv1@4 as nv1, nv2@5 as nv2, rn1@6 as rn1, rn2@7 as rn2, rank1@8 as rank1, rank2@9 as rank2, dense_rank1@10 as dense_rank1, dense_rank2@11 as dense_rank2, lag1@12 as lag1, lag2@13 as lag2, lead1@14 as lead1, lead2@15 as lead2, fvr1@16 as fvr1, fvr2@17 as fvr2, lvr1@18 as lvr1, lvr2@19 as lvr2, lagr1@20 as lagr1, lagr2@21 as lagr2, leadr1@22 as leadr1, leadr2@23 as leadr2]", " GlobalLimitExec: skip=0, fetch=5", " SortExec: fetch=5, expr=[ts@24 DESC]", - " ProjectionExec: expr=[FIRST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@10 as fv1, FIRST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as fv2, LAST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@12 as lv1, LAST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as lv2, NTH_VALUE(annotated_data.inc_col,Int64(5)) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as nv1, NTH_VALUE(annotated_data.inc_col,Int64(5)) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as nv2, ROW_NUMBER() ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@16 as rn1, ROW_NUMBER() ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@17 as rn2, RANK() ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as rank1, RANK() ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as rank2, DENSE_RANK() ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@20 as dense_rank1, DENSE_RANK() ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@21 as dense_rank2, LAG(annotated_data.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@22 as lag1, LAG(annotated_data.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@23 as lag2, LEAD(annotated_data.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as lead1, LEAD(annotated_data.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@25 as lead2, FIRST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as fvr1, FIRST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as fvr2, LAST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@4 as lvr1, LAST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as lvr2, LAG(annotated_data.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@6 as lagr1, LAG(annotated_data.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@7 as lagr2, LEAD(annotated_data.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@8 as leadr1, LEAD(annotated_data.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@9 as leadr2, ts@0 as ts]", + " ProjectionExec: expr=[FIRST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as fv1, FIRST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@12 as fv2, LAST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as lv1, LAST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as lv2, NTH_VALUE(annotated_data.inc_col,Int64(5)) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as nv1, NTH_VALUE(annotated_data.inc_col,Int64(5)) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@16 as nv2, ROW_NUMBER() ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@17 as rn1, ROW_NUMBER() ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@18 as rn2, RANK() ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@19 as rank1, RANK() ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@20 as rank2, DENSE_RANK() ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@21 as dense_rank1, DENSE_RANK() ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@22 as dense_rank2, LAG(annotated_data.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@23 as lag1, LAG(annotated_data.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@24 as lag2, LEAD(annotated_data.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@25 as lead1, LEAD(annotated_data.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@26 as lead2, FIRST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as fvr1, FIRST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@4 as fvr2, LAST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as lvr1, LAST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@6 as lvr2, LAG(annotated_data.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@7 as lagr1, LAG(annotated_data.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@8 as lagr2, LEAD(annotated_data.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@9 as leadr1, LEAD(annotated_data.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@10 as leadr2, ts@0 as ts]", " BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data.inc_col): Ok(Field { name: \"FIRST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, FIRST_VALUE(annotated_data.inc_col): Ok(Field { name: \"FIRST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LAST_VALUE(annotated_data.inc_col): Ok(Field { name: \"LAST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LAST_VALUE(annotated_data.inc_col): Ok(Field { name: \"LAST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, NTH_VALUE(annotated_data.inc_col,Int64(5)): Ok(Field { name: \"NTH_VALUE(annotated_data.inc_col,Int64(5))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, NTH_VALUE(annotated_data.inc_col,Int64(5)): Ok(Field { name: \"NTH_VALUE(annotated_data.inc_col,Int64(5))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, RANK(): Ok(Field { name: \"RANK()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, RANK(): Ok(Field { name: \"RANK()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, DENSE_RANK(): Ok(Field { name: \"DENSE_RANK()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, DENSE_RANK(): Ok(Field { name: \"DENSE_RANK()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LAG(annotated_data.inc_col,Int64(1),Int64(1001)): Ok(Field { name: \"LAG(annotated_data.inc_col,Int64(1),Int64(1001))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LAG(annotated_data.inc_col,Int64(2),Int64(1002)): Ok(Field { name: \"LAG(annotated_data.inc_col,Int64(2),Int64(1002))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LEAD(annotated_data.inc_col,Int64(-1),Int64(1001)): Ok(Field { name: \"LEAD(annotated_data.inc_col,Int64(-1),Int64(1001))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LEAD(annotated_data.inc_col,Int64(4),Int64(1004)): Ok(Field { name: \"LEAD(annotated_data.inc_col,Int64(4),Int64(1004))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }]", " BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data.inc_col): Ok(Field { name: \"FIRST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, FIRST_VALUE(annotated_data.inc_col): Ok(Field { name: \"FIRST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LAST_VALUE(annotated_data.inc_col): Ok(Field { name: \"LAST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LAST_VALUE(annotated_data.inc_col): Ok(Field { name: \"LAST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LAG(annotated_data.inc_col,Int64(1),Int64(1001)): Ok(Field { name: \"LAG(annotated_data.inc_col,Int64(1),Int64(1001))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LAG(annotated_data.inc_col,Int64(2),Int64(1002)): Ok(Field { name: \"LAG(annotated_data.inc_col,Int64(2),Int64(1002))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LEAD(annotated_data.inc_col,Int64(-1),Int64(1001)): Ok(Field { name: \"LEAD(annotated_data.inc_col,Int64(-1),Int64(1001))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LEAD(annotated_data.inc_col,Int64(4),Int64(1004)): Ok(Field { name: \"LEAD(annotated_data.inc_col,Int64(4),Int64(1004))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }]", ] @@ -1239,7 +1241,7 @@ mod tests { "ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, min1@2 as min1, min2@3 as min2, max1@4 as max1, max2@5 as max2, count1@6 as count1, count2@7 as count2, avg1@8 as avg1, avg2@9 as avg2]", " GlobalLimitExec: skip=0, fetch=5", " SortExec: fetch=5, expr=[inc_col@10 ASC NULLS LAST]", - " ProjectionExec: expr=[SUM(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@7 as sum1, SUM(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@2 as sum2, MIN(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@8 as min1, MIN(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as min2, MAX(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@9 as max1, MAX(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as max2, COUNT(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@10 as count1, COUNT(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@5 as count2, AVG(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@11 as avg1, AVG(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@6 as avg2, inc_col@1 as inc_col]", + " ProjectionExec: expr=[SUM(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@8 as sum1, SUM(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, MIN(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@9 as min1, MIN(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as min2, MAX(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@10 as max1, MAX(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@5 as max2, COUNT(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@11 as count1, COUNT(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@6 as count2, AVG(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@12 as avg1, AVG(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@7 as avg2, inc_col@1 as inc_col]", " BoundedWindowAggExec: wdw=[SUM(annotated_data.inc_col): Ok(Field { name: \"SUM(annotated_data.inc_col)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)) }, MIN(annotated_data.inc_col): Ok(Field { name: \"MIN(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)) }, MAX(annotated_data.inc_col): Ok(Field { name: \"MAX(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)) }, COUNT(annotated_data.inc_col): Ok(Field { name: \"COUNT(annotated_data.inc_col)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)) }, AVG(annotated_data.inc_col): Ok(Field { name: \"AVG(annotated_data.inc_col)\", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)) }]", " BoundedWindowAggExec: wdw=[SUM(annotated_data.inc_col): Ok(Field { name: \"SUM(annotated_data.inc_col)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)) }, MIN(annotated_data.inc_col): Ok(Field { name: \"MIN(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)) }, MAX(annotated_data.inc_col): Ok(Field { name: \"MAX(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)) }, COUNT(annotated_data.inc_col): Ok(Field { name: \"COUNT(annotated_data.inc_col)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)) }, AVG(annotated_data.inc_col): Ok(Field { name: \"AVG(annotated_data.inc_col)\", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)) }]", ] @@ -1293,7 +1295,7 @@ mod tests { "ProjectionExec: expr=[first_value1@0 as first_value1, first_value2@1 as first_value2, last_value1@2 as last_value1, last_value2@3 as last_value2, nth_value1@4 as nth_value1]", " GlobalLimitExec: skip=0, fetch=5", " SortExec: fetch=5, expr=[inc_col@5 ASC NULLS LAST]", - " ProjectionExec: expr=[FIRST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@4 as first_value1, FIRST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@2 as first_value2, LAST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as last_value1, LAST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as last_value2, NTH_VALUE(annotated_data.inc_col,Int64(2)) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as nth_value1, inc_col@1 as inc_col]", + " ProjectionExec: expr=[FIRST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as first_value1, FIRST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as first_value2, LAST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as last_value1, LAST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as last_value2, NTH_VALUE(annotated_data.inc_col,Int64(2)) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@7 as nth_value1, inc_col@1 as inc_col]", " BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data.inc_col): Ok(Field { name: \"FIRST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }, LAST_VALUE(annotated_data.inc_col): Ok(Field { name: \"LAST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }, NTH_VALUE(annotated_data.inc_col,Int64(2)): Ok(Field { name: \"NTH_VALUE(annotated_data.inc_col,Int64(2))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }]", " BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data.inc_col): Ok(Field { name: \"FIRST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }, LAST_VALUE(annotated_data.inc_col): Ok(Field { name: \"LAST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }]", ] diff --git a/datafusion/optimizer/src/eliminate_project.rs b/datafusion/optimizer/src/eliminate_project.rs index 143004aaabeb..5c43b8d12cb6 100644 --- a/datafusion/optimizer/src/eliminate_project.rs +++ b/datafusion/optimizer/src/eliminate_project.rs @@ -77,7 +77,7 @@ impl OptimizerRule for EliminateProjection { } } -fn can_eliminate(projection: &Projection, schema: &DFSchemaRef) -> bool { +pub(crate) fn can_eliminate(projection: &Projection, schema: &DFSchemaRef) -> bool { if projection.expr.len() != schema.fields().len() { return false; } diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 2b27b6702d3c..64a1750ee2e4 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -228,7 +228,7 @@ impl Optimizer { Arc::new(PropagateEmptyRelation::new()), Arc::new(FilterNullJoinKeys::default()), Arc::new(EliminateOuterJoin::new()), - // Filters can't be pushed down past Limits, we should do PushDownFilter after LimitPushDown + // Filters can't be pushed down past Limits, we should do PushDownFilter after PushDownLimit Arc::new(PushDownLimit::new()), Arc::new(PushDownFilter::new()), Arc::new(SingleDistinctToGroupBy::new()), @@ -239,6 +239,8 @@ impl Optimizer { Arc::new(CommonSubexprEliminate::new()), Arc::new(PushDownProjection::new()), Arc::new(EliminateProjection::new()), + // PushDownProjection can pushdown Projections through Limits, do PushDownLimit again. + Arc::new(PushDownLimit::new()), ]; Self::with_rules(rules) diff --git a/datafusion/optimizer/src/push_down_projection.rs b/datafusion/optimizer/src/push_down_projection.rs index b6ba3131c9a7..6d7ab481b83a 100644 --- a/datafusion/optimizer/src/push_down_projection.rs +++ b/datafusion/optimizer/src/push_down_projection.rs @@ -18,21 +18,21 @@ //! Projection Push Down optimizer rule ensures that only referenced columns are //! loaded into memory +use crate::eliminate_project::can_eliminate; +use crate::optimizer::ApplyOrder; +use crate::push_down_filter::replace_cols_by_name; use crate::{OptimizerConfig, OptimizerRule}; -use arrow::datatypes::Field; use arrow::error::Result as ArrowResult; +use datafusion_common::ScalarValue::UInt8; use datafusion_common::{ Column, DFField, DFSchema, DFSchemaRef, DataFusionError, Result, ToDFSchema, }; -use datafusion_expr::utils::grouping_set_to_exprlist; +use datafusion_expr::expr::AggregateFunction; +use datafusion_expr::utils::exprlist_to_fields; use datafusion_expr::{ - logical_plan::{ - builder::{build_join_schema, LogicalPlanBuilder}, - Aggregate, Analyze, Join, LogicalPlan, Projection, SubqueryAlias, TableScan, - Union, Window, - }, - utils::{expr_to_columns, exprlist_to_columns, find_sort_exprs, from_plan}, - Expr, + logical_plan::{Aggregate, LogicalPlan, Projection, TableScan, Union}, + utils::{expr_to_columns, exprlist_to_columns}, + Expr, LogicalPlanBuilder, SubqueryAlias, }; use std::collections::HashMap; use std::{ @@ -40,6 +40,18 @@ use std::{ sync::Arc, }; +// if projection is empty return projection-new_plan, else return new_plan. +#[macro_export] +macro_rules! generate_plan { + ($projection_is_empty:expr, $plan:expr, $new_plan:expr) => { + if $projection_is_empty { + $new_plan + } else { + $plan.with_new_inputs(&[$new_plan])? + } + }; +} + /// Optimizer that removes unused projections and aggregations from plans /// This reduces both scans and #[derive(Default)] @@ -49,441 +61,427 @@ impl OptimizerRule for PushDownProjection { fn try_optimize( &self, plan: &LogicalPlan, - config: &dyn OptimizerConfig, + _config: &dyn OptimizerConfig, ) -> Result> { - // set of all columns referred by the plan (and thus considered required by the root) - let required_columns = plan - .schema() - .fields() - .iter() - .map(|f| f.qualified_column()) - .collect::>(); - Ok(Some(optimize_plan( - self, - plan, - &required_columns, - false, - config, - )?)) - } - - fn name(&self) -> &str { - "push_down_projection" - } -} + let projection = match plan { + LogicalPlan::Projection(projection) => projection, + LogicalPlan::Aggregate(agg) => { + let mut required_columns = HashSet::new(); + for e in agg.aggr_expr.iter().chain(agg.group_expr.iter()) { + expr_to_columns(e, &mut required_columns)? + } + let new_expr = get_expr(&required_columns, agg.input.schema())?; + let projection = LogicalPlan::Projection(Projection::try_new( + new_expr, + agg.input.clone(), + )?); + let optimized_child = self + .try_optimize(&projection, _config)? + .unwrap_or(projection); + return Ok(Some(plan.with_new_inputs(&[optimized_child])?)); + } + LogicalPlan::TableScan(scan) if scan.projection.is_none() => { + return Ok(Some(push_down_scan(&HashSet::new(), scan, false)?)); + } + _ => return Ok(None), + }; -impl PushDownProjection { - #[allow(missing_docs)] - pub fn new() -> Self { - Self {} - } -} + let child_plan = &*projection.input; + let projection_is_empty = projection.expr.is_empty(); -/// Recursively transverses the logical plan removing expressions and that are not needed. -fn optimize_plan( - _optimizer: &PushDownProjection, - plan: &LogicalPlan, - required_columns: &HashSet, // set of columns required up to this step - has_projection: bool, - _config: &dyn OptimizerConfig, -) -> Result { - let mut new_required_columns = required_columns.clone(); - let new_plan = match plan { - LogicalPlan::Projection(Projection { - input, - expr, - schema, - .. - }) => { - // projection: - // * remove any expression that is not required - // * construct the new set of required columns - - let mut new_expr = Vec::new(); - let mut new_fields = Vec::new(); - // When meet projection, its expr must contain all columns that its child need. - // So we need create a empty required_columns instead use original new_required_columns. - // Otherwise it cause redundant columns. - let mut new_required_columns = HashSet::new(); - - // Gather all columns needed for expressions in this Projection - schema.fields().iter().enumerate().for_each(|(i, field)| { - if required_columns.contains(&field.qualified_column()) { - new_expr.push(expr[i].clone()); - new_fields.push(field.clone()); + let new_plan = match child_plan { + LogicalPlan::Projection(child_projection) => { + // merge projection + let replace_map = collect_projection_expr(child_projection); + let new_exprs = projection + .expr + .iter() + .map(|expr| replace_cols_by_name(expr.clone(), &replace_map)) + .enumerate() + .map(|(i, e)| match e { + Ok(e) => { + let parent_expr = + projection.schema.fields()[i].qualified_name(); + if e.display_name()? == parent_expr { + Ok(e) + } else { + Ok(e.alias(parent_expr)) + } + } + Err(e) => Err(e), + }) + .collect::>>()?; + let new_plan = LogicalPlan::Projection(Projection::try_new_with_schema( + new_exprs, + child_projection.input.clone(), + projection.schema.clone(), + )?); + + self.try_optimize(&new_plan, _config)?.unwrap_or(new_plan) + } + LogicalPlan::Join(join) => { + // collect column in on/filter in join and projection. + let mut push_columns: HashSet = HashSet::new(); + for e in projection.expr.iter() { + expr_to_columns(e, &mut push_columns)?; + } + for (l, r) in join.on.iter() { + expr_to_columns(l, &mut push_columns)?; + expr_to_columns(r, &mut push_columns)?; + } + if let Some(expr) = &join.filter { + expr_to_columns(expr, &mut push_columns)?; } - }); - for e in new_expr.iter() { - expr_to_columns(e, &mut new_required_columns)? + let new_left = generate_projection( + &push_columns, + join.left.schema(), + join.left.clone(), + )?; + let new_right = generate_projection( + &push_columns, + join.right.schema(), + join.right.clone(), + )?; + let new_join = child_plan.with_new_inputs(&[new_left, new_right])?; + + generate_plan!(projection_is_empty, plan, new_join) + } + LogicalPlan::CrossJoin(join) => { + // collect column in on/filter in join and projection. + let mut push_columns: HashSet = HashSet::new(); + for e in projection.expr.iter() { + expr_to_columns(e, &mut push_columns)?; + } + let new_left = generate_projection( + &push_columns, + join.left.schema(), + join.left.clone(), + )?; + let new_right = generate_projection( + &push_columns, + join.right.schema(), + join.right.clone(), + )?; + let new_join = child_plan.with_new_inputs(&[new_left, new_right])?; + + generate_plan!(projection_is_empty, plan, new_join) } + LogicalPlan::TableScan(scan) + if !scan.projected_schema.fields().is_empty() => + { + let mut used_columns: HashSet = HashSet::new(); + // filter expr may not exist in expr in projection. + // like: TableScan: t1 projection=[bool_col, int_col], full_filters=[t1.id = Int32(1)] + // projection=[bool_col, int_col] don't contain `ti.id`. + exprlist_to_columns(&scan.filters, &mut used_columns)?; + if projection_is_empty { + used_columns + .insert(scan.projected_schema.fields()[0].qualified_column()); + push_down_scan(&used_columns, scan, true)? + } else { + for expr in projection.expr.iter() { + expr_to_columns(expr, &mut used_columns)?; + } + let new_scan = push_down_scan(&used_columns, scan, true)?; - let new_input = - optimize_plan(_optimizer, input, &new_required_columns, true, _config)?; + plan.with_new_inputs(&[new_scan])? + } + } + LogicalPlan::Values(values) if projection_is_empty => { + let first_col = + Expr::Column(values.schema.fields()[0].qualified_column()); + LogicalPlan::Projection(Projection::try_new( + vec![first_col], + Arc::new(child_plan.clone()), + )?) + } + LogicalPlan::Union(union) => { + let mut required_columns = HashSet::new(); + exprlist_to_columns(&projection.expr, &mut required_columns)?; + // When there is no projection, we need to add the first column to the projection + // Because if push empty down, children may output different columns. + if required_columns.is_empty() { + required_columns.insert(union.schema.fields()[0].qualified_column()); + } + // we don't push down projection expr, we just prune columns, so we just push column + // because push expr may cause more cost. + let projection_column_exprs = get_expr(&required_columns, &union.schema)?; + let mut inputs = Vec::with_capacity(union.inputs.len()); + for input in &union.inputs { + let mut replace_map = HashMap::new(); + for (i, field) in input.schema().fields().iter().enumerate() { + replace_map.insert( + union.schema.fields()[i].qualified_name(), + Expr::Column(field.qualified_column()), + ); + } - let new_required_columns_optimized = new_input - .schema() - .fields() - .iter() - .map(|f| f.qualified_column()) - .collect::>(); + let exprs = projection_column_exprs + .iter() + .map(|expr| replace_cols_by_name(expr.clone(), &replace_map)) + .collect::>>()?; - let all_column_exprs = new_expr.iter().all(|e| matches!(e, Expr::Column(_))); + inputs.push(Arc::new(LogicalPlan::Projection(Projection::try_new( + exprs, + input.clone(), + )?))) + } + // create schema of all used columns + let schema = DFSchema::new_with_metadata( + exprlist_to_fields(&projection_column_exprs, child_plan)?, + union.schema.metadata().clone(), + )?; + let new_union = LogicalPlan::Union(Union { + inputs, + schema: Arc::new(schema), + }); + + generate_plan!(projection_is_empty, plan, new_union) + } + LogicalPlan::SubqueryAlias(subquery_alias) => { + let replace_map = generate_column_replace_map(subquery_alias); + let mut required_columns = HashSet::new(); + exprlist_to_columns(&projection.expr, &mut required_columns)?; - if new_fields.is_empty() - || (has_projection - && all_column_exprs - && &new_required_columns_optimized == required_columns) - { - // no need for an expression at all - Ok(new_input) - } else { - let metadata = new_input.schema().metadata().clone(); - Ok(LogicalPlan::Projection(Projection::try_new_with_schema( + let new_required_columns = required_columns + .iter() + .map(|c| { + replace_map.get(c).cloned().ok_or_else(|| { + DataFusionError::Internal("replace column failed".to_string()) + }) + }) + .collect::>>()?; + + let new_expr = + get_expr(&new_required_columns, subquery_alias.input.schema())?; + let new_projection = LogicalPlan::Projection(Projection::try_new( new_expr, - Arc::new(new_input), - DFSchemaRef::new(DFSchema::new_with_metadata(new_fields, metadata)?), - )?)) - } - } - LogicalPlan::Join(Join { - left, - right, - on, - filter, - join_type, - join_constraint, - null_equals_null, - .. - }) => { - for (l, r) in on { - new_required_columns.extend(l.to_columns()?); - new_required_columns.extend(r.to_columns()?); - } + subquery_alias.input.clone(), + )?); + let new_alias = child_plan.with_new_inputs(&[new_projection])?; - if let Some(expr) = filter { - expr_to_columns(expr, &mut new_required_columns)?; + generate_plan!(projection_is_empty, plan, new_alias) } - - let optimized_left = Arc::new(optimize_plan( - _optimizer, - left, - &new_required_columns, - true, - _config, - )?); - - let optimized_right = Arc::new(optimize_plan( - _optimizer, - right, - &new_required_columns, - true, - _config, - )?); - - let schema = build_join_schema( - optimized_left.schema(), - optimized_right.schema(), - join_type, - )?; - - Ok(LogicalPlan::Join(Join { - left: optimized_left, - right: optimized_right, - join_type: *join_type, - join_constraint: *join_constraint, - on: on.clone(), - filter: filter.clone(), - schema: DFSchemaRef::new(schema), - null_equals_null: *null_equals_null, - })) - } - LogicalPlan::Window(Window { - window_expr, input, .. - }) => { - // Gather all columns needed for expressions in this Window - let mut new_window_expr = Vec::new(); - { - window_expr.iter().try_for_each(|expr| { - let name = &expr.display_name()?; - let column = Column::from_name(name); + LogicalPlan::Aggregate(agg) => { + let mut required_columns = HashSet::new(); + exprlist_to_columns(&projection.expr, &mut required_columns)?; + // Gather all columns needed for expressions in this Aggregate + let mut new_aggr_expr = vec![]; + for e in agg.aggr_expr.iter() { + let column = Column::from_name(e.display_name()?); if required_columns.contains(&column) { - new_window_expr.push(expr.clone()); - new_required_columns.insert(column); - // add to the new set of required columns - expr_to_columns(expr, &mut new_required_columns) - } else { - Ok(()) + new_aggr_expr.push(e.clone()); + } + } + + // if new_aggr_expr emtpy and aggr is COUNT(UInt8(1)), push it + if new_aggr_expr.is_empty() && agg.aggr_expr.len() == 1 { + if let Expr::AggregateFunction(AggregateFunction { + fun, args, .. + }) = &agg.aggr_expr[0] + { + if matches!(fun, datafusion_expr::AggregateFunction::Count) + && args.len() == 1 + && args[0] == Expr::Literal(UInt8(Some(1))) + { + new_aggr_expr.push(agg.aggr_expr[0].clone()); + } } - })?; + } + + let new_agg = LogicalPlan::Aggregate(Aggregate::try_new( + agg.input.clone(), + agg.group_expr.clone(), + new_aggr_expr, + )?); + + generate_plan!(projection_is_empty, plan, new_agg) } + LogicalPlan::Window(window) => { + let mut required_columns = HashSet::new(); + exprlist_to_columns(&projection.expr, &mut required_columns)?; + // Gather all columns needed for expressions in this Window + let mut new_window_expr = vec![]; + for e in window.window_expr.iter() { + let column = Column::from_name(e.display_name()?); + if required_columns.contains(&column) { + new_window_expr.push(e.clone()); + } + } - // none columns in window expr are needed, remove the window expr - if new_window_expr.is_empty() { - return LogicalPlanBuilder::from(optimize_plan( - _optimizer, - input, - required_columns, - true, - _config, - )?) - .build(); - }; + if new_window_expr.is_empty() { + // none columns in window expr are needed, remove the window expr + let new_window = window.input.as_ref().clone(); - // for all the retained window expr, find their sort expressions if any, and retain these - exprlist_to_columns( - &find_sort_exprs(&new_window_expr), - &mut new_required_columns, - )?; - - LogicalPlanBuilder::from(optimize_plan( - _optimizer, - input, - &new_required_columns, - true, - _config, - )?) - .window(new_window_expr)? - .build() - } - LogicalPlan::Aggregate(Aggregate { - group_expr, - aggr_expr, - input, - .. - }) => { - // aggregate: - // * remove any aggregate expression that is not required - // * construct the new set of required columns - - // Find distinct group by exprs in the case where we have a grouping set - let all_group_expr: Vec = grouping_set_to_exprlist(group_expr)?; - exprlist_to_columns(&all_group_expr, &mut new_required_columns)?; - - // Gather all columns needed for expressions in this Aggregate - let mut new_aggr_expr = Vec::new(); - aggr_expr.iter().try_for_each(|expr| { - let name = &expr.display_name()?; - let column = Column::from_name(name); - if required_columns.contains(&column) { - new_aggr_expr.push(expr.clone()); - new_required_columns.insert(column); - - // add to the new set of required columns - expr_to_columns(expr, &mut new_required_columns) + generate_plan!(projection_is_empty, plan, new_window) } else { - Ok(()) - } - })?; - - Ok(LogicalPlan::Aggregate(Aggregate::try_new( - Arc::new(optimize_plan( - _optimizer, - input, - &new_required_columns, - true, - _config, - )?), - group_expr.clone(), - new_aggr_expr, - )?)) - } - // scans: - // * remove un-used columns from the scan projection - LogicalPlan::TableScan(scan) => { - // filter expr may not exist in expr in projection. - // like: TableScan: t1 projection=[bool_col, int_col], full_filters=[t1.id = Int32(1)] - // projection=[bool_col, int_col] don't contain `ti.id`. - exprlist_to_columns(&scan.filters, &mut new_required_columns)?; - push_down_scan(scan, &new_required_columns, has_projection) - } - LogicalPlan::Explain { .. } => Err(DataFusionError::Internal( - "Unsupported logical plan: Explain must be root of the plan".to_string(), - )), - LogicalPlan::Analyze(a) => { - // make sure we keep all the columns from the input plan - let required_columns = a - .input - .schema() - .fields() - .iter() - .map(|f| f.qualified_column()) - .collect::>(); - - Ok(LogicalPlan::Analyze(Analyze { - input: Arc::new(optimize_plan( - _optimizer, - &a.input, - &required_columns, - false, - _config, - )?), - verbose: a.verbose, - schema: a.schema.clone(), - })) - } - LogicalPlan::Union(Union { inputs, schema }) => { - // UNION inputs will reference the same column with different identifiers, so we need - // to populate new_required_columns by unqualified column name based on required fields - // from the resulting UNION output - let union_required_fields = schema - .fields() - .iter() - .filter(|f| new_required_columns.contains(&f.qualified_column())) - .map(|f| f.field()) - .collect::>(); - let new_inputs = inputs - .iter() - .map(|input_plan| { - input_plan - .schema() - .fields() - .iter() - .filter(|f| union_required_fields.contains(f.field())) - .for_each(|f| { - new_required_columns.insert(f.qualified_column()); - }); - optimize_plan( - _optimizer, - input_plan, - &new_required_columns, - has_projection, - _config, - ) - }) - .collect::>>()?; - let new_schema = DFSchema::new_with_metadata( - schema - .fields() - .iter() - .filter(|f| union_required_fields.contains(f.field())) - .cloned() - .collect(), - schema.metadata().clone(), - )?; - Ok(LogicalPlan::Union(Union { - inputs: new_inputs.iter().cloned().map(Arc::new).collect(), - schema: Arc::new(new_schema), - })) - } - LogicalPlan::SubqueryAlias(SubqueryAlias { input, alias, .. }) => { - let new_required_columns = - replace_alias(required_columns, alias, input.schema()); - let child = optimize_plan( - _optimizer, - input, - &new_required_columns, - has_projection, - _config, - )?; - from_plan(plan, &plan.expressions(), &[child]) - } - // at a distinct, all columns are required - LogicalPlan::Distinct(distinct) => { - let new_required_columns = distinct - .input - .schema() - .fields() - .iter() - .map(|f| f.qualified_column()) - .collect(); - let child = optimize_plan( - _optimizer, - distinct.input.as_ref(), - &new_required_columns, - has_projection, - _config, - )?; - from_plan(plan, &[], &[child]) - } - // all other nodes: Add any additional columns used by - // expressions in this node to the list of required columns - LogicalPlan::Limit(_) - | LogicalPlan::Filter { .. } - | LogicalPlan::Repartition(_) - | LogicalPlan::EmptyRelation(_) - | LogicalPlan::Subquery(_) - | LogicalPlan::Values(_) - | LogicalPlan::Sort { .. } - | LogicalPlan::CreateExternalTable(_) - | LogicalPlan::CreateMemoryTable(_) - | LogicalPlan::CreateView(_) - | LogicalPlan::CreateCatalogSchema(_) - | LogicalPlan::CreateCatalog(_) - | LogicalPlan::DropTable(_) - | LogicalPlan::DropView(_) - | LogicalPlan::SetVariable(_) - | LogicalPlan::DescribeTable(_) - | LogicalPlan::CrossJoin(_) - | LogicalPlan::Dml(_) - | LogicalPlan::Unnest(_) - | LogicalPlan::Extension { .. } - | LogicalPlan::Prepare(_) => { - let expr = plan.expressions(); - // collect all required columns by this plan - exprlist_to_columns(&expr, &mut new_required_columns)?; - - // apply the optimization to all inputs of the plan - let inputs = plan.inputs(); - let new_inputs = inputs - .iter() - .map(|input_plan| { - optimize_plan( - _optimizer, - input_plan, - &new_required_columns, - has_projection, - _config, - ) - }) - .collect::>>()?; - - from_plan(plan, &expr, &new_inputs) - } - }; + let new_window = LogicalPlanBuilder::from((*(window.input)).clone()) + .window(new_window_expr)? + .build()?; - // when this rule is applied multiple times it will insert duplicate nested projections, - // so we catch this here - let with_dupe_projection_removed = match new_plan? { - LogicalPlan::Projection(p) => match p.input.as_ref() { - LogicalPlan::Projection(p2) if projection_equal(&p, p2) => { - LogicalPlan::Projection(p2.clone()) + generate_plan!(projection_is_empty, plan, new_window) + } } - _ => LogicalPlan::Projection(p), - }, - other => other, - }; + LogicalPlan::Filter(filter) => { + if can_eliminate(projection, child_plan.schema()) { + // when projection schema == filter schema, we can commute directly. + let new_proj = + plan.with_new_inputs(&[filter.input.as_ref().clone()])?; + child_plan.with_new_inputs(&[new_proj])? + } else { + let mut required_columns = HashSet::new(); + exprlist_to_columns(&projection.expr, &mut required_columns)?; + exprlist_to_columns( + &[filter.predicate.clone()], + &mut required_columns, + )?; + + let new_expr = get_expr(&required_columns, filter.input.schema())?; + let new_projection = LogicalPlan::Projection(Projection::try_new( + new_expr, + filter.input.clone(), + )?); + let new_filter = child_plan.with_new_inputs(&[new_projection])?; + + generate_plan!(projection_is_empty, plan, new_filter) + } + } + LogicalPlan::Sort(sort) => { + if can_eliminate(projection, child_plan.schema()) { + // can commute + let new_proj = plan.with_new_inputs(&[(*sort.input).clone()])?; + child_plan.with_new_inputs(&[new_proj])? + } else { + let mut required_columns = HashSet::new(); + exprlist_to_columns(&projection.expr, &mut required_columns)?; + exprlist_to_columns(&sort.expr, &mut required_columns)?; + + let new_expr = get_expr(&required_columns, sort.input.schema())?; + let new_projection = LogicalPlan::Projection(Projection::try_new( + new_expr, + sort.input.clone(), + )?); + let new_sort = child_plan.with_new_inputs(&[new_projection])?; + + generate_plan!(projection_is_empty, plan, new_sort) + } + } + LogicalPlan::Limit(limit) => { + // can commute + let new_proj = plan.with_new_inputs(&[limit.input.as_ref().clone()])?; + child_plan.with_new_inputs(&[new_proj])? + } + _ => return Ok(None), + }; + + Ok(Some(new_plan)) + } + + fn name(&self) -> &str { + "push_down_projection" + } + + fn apply_order(&self) -> Option { + Some(ApplyOrder::TopDown) + } +} - Ok(with_dupe_projection_removed) +impl PushDownProjection { + #[allow(missing_docs)] + pub fn new() -> Self { + Self {} + } } -fn projection_equal(p: &Projection, p2: &Projection) -> bool { - p.expr.len() == p2.expr.len() && p.expr.iter().zip(&p2.expr).all(|(l, r)| l == r) +fn generate_column_replace_map( + subquery_alias: &SubqueryAlias, +) -> HashMap { + subquery_alias + .input + .schema() + .fields() + .iter() + .enumerate() + .map(|(i, field)| { + ( + subquery_alias.schema.fields()[i].qualified_column(), + field.qualified_column(), + ) + }) + .collect() } -fn replace_alias( - required_columns: &HashSet, - alias: &str, - input_schema: &DFSchemaRef, -) -> HashSet { - let mut map = HashMap::new(); - for field in input_schema.fields() { - let col = field.qualified_column(); - let alias_col = Column { - relation: Some(alias.to_owned()), - name: col.name.clone(), - }; - map.insert(alias_col, col); +pub fn collect_projection_expr(projection: &Projection) -> HashMap { + projection + .schema + .fields() + .iter() + .enumerate() + .flat_map(|(i, field)| { + // strip alias, as they should not be part of filters + let expr = match &projection.expr[i] { + Expr::Alias(expr, _) => expr.as_ref().clone(), + expr => expr.clone(), + }; + + // Convert both qualified and unqualified fields + [ + (field.name().clone(), expr.clone()), + (field.qualified_name(), expr), + ] + }) + .collect::>() +} + +// Get the projection exprs from columns in the order of the schema +fn get_expr(columns: &HashSet, schema: &DFSchemaRef) -> Result> { + let expr = schema + .fields() + .iter() + .flat_map(|field| { + let qc = field.qualified_column(); + let uqc = field.unqualified_column(); + if columns.contains(&qc) || columns.contains(&uqc) { + Some(Expr::Column(qc)) + } else { + None + } + }) + .collect::>(); + if columns.len() != expr.len() { + Err(DataFusionError::Plan(format!( + "required columns can't push down, columns: {columns:?}" + ))) + } else { + Ok(expr) } - required_columns +} + +fn generate_projection( + used_columns: &HashSet, + schema: &DFSchemaRef, + input: Arc, +) -> Result { + let expr = schema + .fields() .iter() - .map(|col| map.get(col).unwrap_or(col).clone()) - .collect::>() + .flat_map(|field| { + let column = field.qualified_column(); + if used_columns.contains(&column) { + Some(Expr::Column(column)) + } else { + None + } + }) + .collect::>(); + + Ok(LogicalPlan::Projection(Projection::try_new(expr, input)?)) } fn push_down_scan( + used_columns: &HashSet, scan: &TableScan, - required_columns: &HashSet, has_projection: bool, ) -> Result { // once we reach the table scan, we can use the accumulated set of column @@ -495,7 +493,7 @@ fn push_down_scan( // Use BTreeSet to remove potential duplicates (e.g. union) as // well as to sort the projection to ensure deterministic behavior let schema = scan.source.schema(); - let mut projection: BTreeSet = required_columns + let mut projection: BTreeSet = used_columns .iter() .filter(|c| { c.relation.is_none() || c.relation.as_ref().unwrap() == &scan.table_name @@ -543,7 +541,6 @@ fn push_down_scan( let projected_schema = projected_fields.to_dfschema_ref()?; - // return the table scan with projection Ok(LogicalPlan::TableScan(TableScan { table_name: scan.table_name.clone(), source: scan.source.clone(), @@ -557,9 +554,12 @@ fn push_down_scan( #[cfg(test)] mod tests { use super::*; + use crate::eliminate_project::EliminateProjection; + use crate::optimizer::Optimizer; use crate::test::*; use crate::OptimizerContext; - use arrow::datatypes::{DataType, Schema}; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::DFSchema; use datafusion_expr::expr; use datafusion_expr::expr::Cast; use datafusion_expr::{ @@ -580,9 +580,7 @@ mod tests { let expected = "Aggregate: groupBy=[[]], aggr=[[MAX(test.b)]]\ \n TableScan: test projection=[b]"; - assert_optimized_plan_eq(&plan, expected); - - Ok(()) + assert_optimized_plan_eq(&plan, expected) } #[test] @@ -596,9 +594,7 @@ mod tests { let expected = "Aggregate: groupBy=[[test.c]], aggr=[[MAX(test.b)]]\ \n TableScan: test projection=[b, c]"; - assert_optimized_plan_eq(&plan, expected); - - Ok(()) + assert_optimized_plan_eq(&plan, expected) } #[test] @@ -614,9 +610,7 @@ mod tests { \n SubqueryAlias: a\ \n TableScan: test projection=[b, c]"; - assert_optimized_plan_eq(&plan, expected); - - Ok(()) + assert_optimized_plan_eq(&plan, expected) } #[test] @@ -629,12 +623,11 @@ mod tests { .build()?; let expected = "Aggregate: groupBy=[[]], aggr=[[MAX(test.b)]]\ - \n Filter: test.c > Int32(1)\ - \n TableScan: test projection=[b, c]"; + \n Projection: test.b\ + \n Filter: test.c > Int32(1)\ + \n TableScan: test projection=[b, c]"; - assert_optimized_plan_eq(&plan, expected); - - Ok(()) + assert_optimized_plan_eq(&plan, expected) } #[test] @@ -648,9 +641,7 @@ mod tests { let expected = "Projection: test.a, test.c, test.b\ \n TableScan: test projection=[a, b, c]"; - assert_optimized_plan_eq(&plan, expected); - - Ok(()) + assert_optimized_plan_eq(&plan, expected) } #[test] @@ -660,9 +651,7 @@ mod tests { let plan = table_scan(Some("test"), &schema, Some(vec![1, 0, 2]))?.build()?; let expected = "TableScan: test projection=[b, a, c]"; - assert_optimized_plan_eq(&plan, expected); - - Ok(()) + assert_optimized_plan_eq(&plan, expected) } #[test] @@ -675,9 +664,7 @@ mod tests { let expected = "Projection: test.a, test.b\ \n TableScan: test projection=[b, a]"; - assert_optimized_plan_eq(&plan, expected); - - Ok(()) + assert_optimized_plan_eq(&plan, expected) } #[test] @@ -690,9 +677,7 @@ mod tests { let expected = "Projection: test.c, test.b, test.a\ \n TableScan: test projection=[a, b, c]"; - assert_optimized_plan_eq(&plan, expected); - - Ok(()) + assert_optimized_plan_eq(&plan, expected) } #[test] @@ -710,12 +695,11 @@ mod tests { let expected = "Projection: test.a, test.c, test.b\ \n Filter: test.a > Int32(1)\ \n Filter: test.b > Int32(1)\ - \n Filter: test.c > Int32(1)\ - \n TableScan: test projection=[a, b, c]"; - - assert_optimized_plan_eq(&plan, expected); - - Ok(()) + \n Projection: test.c, test.a, test.b\ + \n Filter: test.c > Int32(1)\ + \n Projection: test.c, test.b, test.a\ + \n TableScan: test projection=[a, b, c]"; + assert_optimized_plan_eq(&plan, expected) } #[test] @@ -731,17 +715,16 @@ mod tests { .build()?; // make sure projections are pushed down to both table scans - let expected = "Projection: test.a, test.b, test2.c1\ - \n Left Join: test.a = test2.c1\ - \n TableScan: test projection=[a, b]\ - \n TableScan: test2 projection=[c1]"; + let expected = "Left Join: test.a = test2.c1\ + \n TableScan: test projection=[a, b]\ + \n TableScan: test2 projection=[c1]"; let optimized_plan = optimize(&plan)?; let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); // make sure schema for join node include both join columns - let optimized_join = optimized_plan.inputs()[0]; + let optimized_join = optimized_plan; assert_eq!( **optimized_join.schema(), DFSchema::new_with_metadata( @@ -855,29 +838,23 @@ mod tests { let expected = "Projection: CAST(test.c AS Float64)\ \n TableScan: test projection=[c]"; - assert_optimized_plan_eq(&projection, expected); - - Ok(()) + assert_optimized_plan_eq(&projection, expected) } #[test] fn table_scan_projected_schema() -> Result<()> { let table_scan = test_table_scan()?; - assert_eq!(3, table_scan.schema().fields().len()); - assert_fields_eq(&table_scan, vec!["a", "b", "c"]); - - let plan = LogicalPlanBuilder::from(table_scan) + let plan = LogicalPlanBuilder::from(test_table_scan()?) .project(vec![col("a"), col("b")])? .build()?; + assert_eq!(3, table_scan.schema().fields().len()); + assert_fields_eq(&table_scan, vec!["a", "b", "c"]); assert_fields_eq(&plan, vec!["a", "b"]); - let expected = "Projection: test.a, test.b\ - \n TableScan: test projection=[a, b]"; - - assert_optimized_plan_eq(&plan, expected); + let expected = "TableScan: test projection=[a, b]"; - Ok(()) + assert_optimized_plan_eq(&plan, expected) } #[test] @@ -896,12 +873,9 @@ mod tests { assert_fields_eq(&plan, vec!["a", "b"]); - let expected = "Projection: a, b\ - \n TableScan: test projection=[a, b]"; - - assert_optimized_plan_eq(&plan, expected); + let expected = "TableScan: test projection=[a, b]"; - Ok(()) + assert_optimized_plan_eq(&plan, expected) } #[test] @@ -921,9 +895,7 @@ mod tests { \n Projection: test.c, test.a\ \n TableScan: test projection=[a, c]"; - assert_optimized_plan_eq(&plan, expected); - - Ok(()) + assert_optimized_plan_eq(&plan, expected) } #[test] @@ -932,8 +904,7 @@ mod tests { let plan = LogicalPlanBuilder::from(table_scan).build()?; // should expand projection to all columns without projection let expected = "TableScan: test projection=[a, b, c]"; - assert_optimized_plan_eq(&plan, expected); - Ok(()) + assert_optimized_plan_eq(&plan, expected) } #[test] @@ -944,8 +915,7 @@ mod tests { .build()?; let expected = "Projection: Int64(1), Int64(2)\ \n TableScan: test projection=[a]"; - assert_optimized_plan_eq(&plan, expected); - Ok(()) + assert_optimized_plan_eq(&plan, expected) } /// tests that it removes unused columns in projections @@ -964,15 +934,14 @@ mod tests { assert_fields_eq(&plan, vec!["c", "MAX(test.a)"]); + let plan = optimize(&plan).expect("failed to optimize plan"); let expected = "\ Aggregate: groupBy=[[test.c]], aggr=[[MAX(test.a)]]\ \n Filter: test.c > Int32(1)\ \n Projection: test.c, test.a\ \n TableScan: test projection=[a, c]"; - assert_optimized_plan_eq(&plan, expected); - - Ok(()) + assert_optimized_plan_eq(&plan, expected) } /// tests that it removes un-needed projections @@ -994,9 +963,7 @@ mod tests { Projection: Int32(1) AS a\ \n TableScan: test projection=[a]"; - assert_optimized_plan_eq(&plan, expected); - - Ok(()) + assert_optimized_plan_eq(&plan, expected) } /// tests that optimizing twice yields same plan @@ -1040,9 +1007,7 @@ mod tests { \n Aggregate: groupBy=[[test.a, test.c]], aggr=[[MAX(test.b)]]\ \n TableScan: test projection=[a, b, c]"; - assert_optimized_plan_eq(&plan, expected); - - Ok(()) + assert_optimized_plan_eq(&plan, expected) } #[test] @@ -1066,9 +1031,7 @@ mod tests { let expected = "Aggregate: groupBy=[[test.a]], aggr=[[COUNT(test.b), COUNT(test.b) FILTER (WHERE c > Int32(42)) AS count2]]\ \n TableScan: test projection=[a, b, c]"; - assert_optimized_plan_eq(&plan, expected); - - Ok(()) + assert_optimized_plan_eq(&plan, expected) } #[test] @@ -1085,19 +1048,35 @@ mod tests { \n Distinct:\ \n TableScan: test projection=[a, b]"; - assert_optimized_plan_eq(&plan, expected); - - Ok(()) + assert_optimized_plan_eq(&plan, expected) } - fn assert_optimized_plan_eq(plan: &LogicalPlan, expected: &str) { + fn assert_optimized_plan_eq(plan: &LogicalPlan, expected: &str) -> Result<()> { let optimized_plan = optimize(plan).expect("failed to optimize plan"); let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); + Ok(()) } fn optimize(plan: &LogicalPlan) -> Result { - let rule = PushDownProjection::new(); - Ok(rule.try_optimize(plan, &OptimizerContext::new())?.unwrap()) + let optimizer = Optimizer::with_rules(vec![ + Arc::new(PushDownProjection::new()), + Arc::new(EliminateProjection::new()), + ]); + let mut optimized_plan = optimizer + .optimize_recursively( + optimizer.rules.get(0).unwrap(), + plan, + &OptimizerContext::new(), + )? + .unwrap_or_else(|| plan.clone()); + optimized_plan = optimizer + .optimize_recursively( + optimizer.rules.get(1).unwrap(), + &optimized_plan, + &OptimizerContext::new(), + )? + .unwrap_or(optimized_plan); + Ok(optimized_plan) } } diff --git a/datafusion/optimizer/tests/integration-test.rs b/datafusion/optimizer/tests/integration-test.rs index b84d15fae735..7e05cffe87aa 100644 --- a/datafusion/optimizer/tests/integration-test.rs +++ b/datafusion/optimizer/tests/integration-test.rs @@ -71,8 +71,9 @@ fn subquery_filter_with_cast() -> Result<()> { \n SubqueryAlias: __scalar_sq_1\ \n Projection: AVG(test.col_int32) AS __value\ \n Aggregate: groupBy=[[]], aggr=[[AVG(test.col_int32)]]\ - \n Filter: test.col_utf8 >= Utf8(\"2002-05-08\") AND test.col_utf8 <= Utf8(\"2002-05-13\")\ - \n TableScan: test projection=[col_int32, col_utf8]"; + \n Projection: test.col_int32\ + \n Filter: test.col_utf8 >= Utf8(\"2002-05-08\") AND test.col_utf8 <= Utf8(\"2002-05-13\")\ + \n TableScan: test projection=[col_int32, col_utf8]"; assert_eq!(expected, format!("{plan:?}")); Ok(()) } diff --git a/datafusion/substrait/tests/roundtrip_logical_plan.rs b/datafusion/substrait/tests/roundtrip_logical_plan.rs index 47ce1bbcd9b6..3ce3343d1431 100644 --- a/datafusion/substrait/tests/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/roundtrip_logical_plan.rs @@ -107,7 +107,7 @@ mod tests { #[tokio::test] async fn simple_distinct() -> Result<()> { test_alias( - "SELECT * FROM (SELECT distinct a FROM data)", // `SELECT *` is used to add `projection` at the root + "SELECT distinct a FROM data", "SELECT a FROM data GROUP BY a", ) .await @@ -116,7 +116,7 @@ mod tests { #[tokio::test] async fn select_distinct_two_fields() -> Result<()> { test_alias( - "SELECT * FROM (SELECT distinct a, b FROM data)", // `SELECT *` is used to add `projection` at the root + "SELECT distinct a, b FROM data", "SELECT a, b FROM data GROUP BY a, b", ) .await