From 1b847e05589ed742c0822e9fa78dc7803175a1a1 Mon Sep 17 00:00:00 2001 From: Edmondo Porcu Date: Fri, 29 Nov 2024 19:52:50 -0500 Subject: [PATCH 1/6] Implementing Unit testing for Python --- .github/workflows/rust.yml | 8 +++++--- {datafusion_ray => scripts}/main.py | 0 {datafusion_ray/tests => tests}/test_context.py | 17 +++++++++++++---- 3 files changed, 18 insertions(+), 7 deletions(-) rename {datafusion_ray => scripts}/main.py (100%) rename {datafusion_ray/tests => tests}/test_context.py (66%) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 3e03704..909441f 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -32,14 +32,14 @@ jobs: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 - name: Install protobuf compiler shell: bash run: sudo apt-get install protobuf-compiler - name: Build Rust code run: cargo build --verbose - name: Set up Python - uses: actions/setup-python@v2 + uses: actions/setup-python@v5 with: python-version: ${{ env.PYTHON_VERSION }} - name: Install test dependencies @@ -49,5 +49,7 @@ jobs: - name: Generate test data run: | ./scripts/gen-test-data.sh - - name: Run tests + - name: Run Rust tests run: cargo test --verbose + - name: Run Python tests + run: python -m pytest diff --git a/datafusion_ray/main.py b/scripts/main.py similarity index 100% rename from datafusion_ray/main.py rename to scripts/main.py diff --git a/datafusion_ray/tests/test_context.py b/tests/test_context.py similarity index 66% rename from datafusion_ray/tests/test_context.py rename to tests/test_context.py index 40b2578..36695c1 100644 --- a/datafusion_ray/tests/test_context.py +++ b/tests/test_context.py @@ -15,12 +15,21 @@ # specific language governing permissions and limitations # under the License. -from datafusion_ray import Context +from datafusion_ray.context import DatafusionRayContext from datafusion import SessionContext +import pytest -def test(): +def test_basic_query_succeed(): df_ctx = SessionContext() - ctx = Context(df_ctx, False) + ctx = DatafusionRayContext(df_ctx) df_ctx.register_csv("tips", "examples/tips.csv", has_header=True) - ctx.plan("SELECT * FROM tips") + record_batch = ctx.sql("SELECT * FROM tips") + assert record_batch.num_rows == 244 + + +def test_no_result_query(): + df_ctx = SessionContext() + ctx = DatafusionRayContext(df_ctx) + df_ctx.register_csv("tips", "examples/tips.csv", has_header=True) + ctx.sql("CREATE VIEW tips_view AS SELECT * FROM tips") From b4aab9a964dd990bc14a97ad44b39dae2e6ae96e Mon Sep 17 00:00:00 2001 From: Edmondo Porcu Date: Sat, 30 Nov 2024 16:01:05 -0500 Subject: [PATCH 2/6] Installing all deps in CI --- .github/workflows/rust.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 909441f..1503cab 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -46,6 +46,7 @@ jobs: run: | python -m pip install --upgrade pip pip install -r tpch/requirements.txt + pip install -r requirements-in.txt - name: Generate test data run: | ./scripts/gen-test-data.sh From b3dddd7968c761272a8f00bd3cf36484f2d49dfd Mon Sep 17 00:00:00 2001 From: Edmondo Porcu Date: Tue, 10 Dec 2024 21:37:05 -0500 Subject: [PATCH 3/6] Adding maturin develop --- .github/workflows/rust.yml | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 1503cab..8620498 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -46,11 +46,15 @@ jobs: run: | python -m pip install --upgrade pip pip install -r tpch/requirements.txt - pip install -r requirements-in.txt - name: Generate test data run: | ./scripts/gen-test-data.sh - name: Run Rust tests run: cargo test --verbose - name: Run Python tests - run: python -m pytest + run: | + python -m venv venv + source venv/bin/activate + pip install -r requirements-in.txt + maturin develop + python -m pytest From b298923000e9d2b6510dc5f3fbdbc34e6edebd96 Mon Sep 17 00:00:00 2001 From: Edmondo Porcu Date: Fri, 13 Dec 2024 17:40:26 -0500 Subject: [PATCH 4/6] Restoring correct input partitioning --- src/query_stage.rs | 7 ++----- tests/test_context.py | 1 - 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/src/query_stage.rs b/src/query_stage.rs index 084cd72..bce824a 100644 --- a/src/query_stage.rs +++ b/src/query_stage.rs @@ -18,7 +18,7 @@ use crate::context::serialize_execution_plan; use crate::shuffle::{ShuffleCodec, ShuffleReaderExec}; use datafusion::error::Result; -use datafusion::physical_plan::{ExecutionPlan, Partitioning}; +use datafusion::physical_plan::{ExecutionPlan, ExecutionPlanProperties, Partitioning}; use datafusion::prelude::SessionContext; use datafusion_proto::bytes::physical_plan_from_bytes_with_extension_codec; use pyo3::prelude::*; @@ -99,10 +99,7 @@ impl QueryStage { /// Get the input partition count. This is the same as the number of concurrent tasks /// when we schedule this query stage for execution pub fn get_input_partition_count(&self) -> usize { - self.plan.children()[0] - .properties() - .output_partitioning() - .partition_count() + self.plan.output_partitioning().partition_count() } pub fn get_output_partition_count(&self) -> usize { diff --git a/tests/test_context.py b/tests/test_context.py index 36695c1..6e1b511 100644 --- a/tests/test_context.py +++ b/tests/test_context.py @@ -17,7 +17,6 @@ from datafusion_ray.context import DatafusionRayContext from datafusion import SessionContext -import pytest def test_basic_query_succeed(): From f07c38d8068afbb20b6aaaab8682f1d62bc6c35a Mon Sep 17 00:00:00 2001 From: Edmondo Porcu Date: Fri, 13 Dec 2024 18:06:49 -0500 Subject: [PATCH 5/6] Generated new plans --- testdata/expected-plans/q1.txt | 2 +- testdata/expected-plans/q10.txt | 4 +- testdata/expected-plans/q11.txt | 10 +- testdata/expected-plans/q12.txt | 71 ------------- testdata/expected-plans/q13.txt | 2 +- testdata/expected-plans/q14.txt | 2 +- testdata/expected-plans/q16.txt | 113 -------------------- testdata/expected-plans/q17.txt | 2 +- testdata/expected-plans/q18.txt | 2 +- testdata/expected-plans/q19.txt | 65 ------------ testdata/expected-plans/q2.txt | 16 +-- testdata/expected-plans/q20.txt | 8 +- testdata/expected-plans/q21.txt | 6 +- testdata/expected-plans/q22.txt | 2 +- testdata/expected-plans/q3.txt | 2 +- testdata/expected-plans/q4.txt | 2 +- testdata/expected-plans/q5.txt | 8 +- testdata/expected-plans/q7.txt | 182 -------------------------------- testdata/expected-plans/q8.txt | 12 +-- testdata/expected-plans/q9.txt | 8 +- 20 files changed, 44 insertions(+), 475 deletions(-) delete mode 100644 testdata/expected-plans/q12.txt delete mode 100644 testdata/expected-plans/q16.txt delete mode 100644 testdata/expected-plans/q19.txt delete mode 100644 testdata/expected-plans/q7.txt diff --git a/testdata/expected-plans/q1.txt b/testdata/expected-plans/q1.txt index 8eaff99..9889d29 100644 --- a/testdata/expected-plans/q1.txt +++ b/testdata/expected-plans/q1.txt @@ -42,7 +42,7 @@ ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_return CoalesceBatchesExec: target_batch_size=8192 ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 2)) -Query Stage #2 (2 -> 1): +Query Stage #2 (1 -> 1): SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 2)) diff --git a/testdata/expected-plans/q10.txt b/testdata/expected-plans/q10.txt index 916dcbb..dd81b58 100644 --- a/testdata/expected-plans/q10.txt +++ b/testdata/expected-plans/q10.txt @@ -60,7 +60,7 @@ SortPreservingMergeExec: [revenue@2 DESC], fetch=20 DataFusion Ray Distributed Plan =========== -Query Stage #0 (1 -> 2): +Query Stage #0 (2 -> 2): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name] @@ -117,7 +117,7 @@ ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "c_custke CoalesceBatchesExec: target_batch_size=8192 ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }, Column { name: "c_name", index: 1 }, Column { name: "c_acctbal", index: 2 }, Column { name: "c_phone", index: 3 }, Column { name: "n_name", index: 4 }, Column { name: "c_address", index: 5 }, Column { name: "c_comment", index: 6 }], 2)) -Query Stage #8 (2 -> 1): +Query Stage #8 (1 -> 1): SortPreservingMergeExec: [revenue@2 DESC], fetch=20 ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }, Column { name: "c_name", index: 1 }, Column { name: "c_acctbal", index: 3 }, Column { name: "c_phone", index: 6 }, Column { name: "n_name", index: 4 }, Column { name: "c_address", index: 5 }, Column { name: "c_comment", index: 7 }], 2)) diff --git a/testdata/expected-plans/q11.txt b/testdata/expected-plans/q11.txt index 4478944..8d822d7 100644 --- a/testdata/expected-plans/q11.txt +++ b/testdata/expected-plans/q11.txt @@ -86,13 +86,13 @@ SortPreservingMergeExec: [value@1 DESC] DataFusion Ray Distributed Plan =========== -Query Stage #0 (1 -> 2): +Query Stage #0 (2 -> 2): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: n_name@1 = ALGERIA, projection=[n_nationkey@0] ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name], predicate=n_name@1 = ALGERIA, pruning_predicate=CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= ALGERIA AND ALGERIA <= n_name_max@1 END, required_guarantees=[n_name in (ALGERIA)] -Query Stage #1 (1 -> 2): +Query Stage #1 (2 -> 2): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_nationkey] @@ -120,13 +120,13 @@ ShuffleWriterExec(stage_id=4, output_partitioning=Hash([], 2)) CoalesceBatchesExec: target_batch_size=8192 ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 2)) -Query Stage #5 (1 -> 2): +Query Stage #5 (2 -> 2): ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: n_name@1 = ALGERIA, projection=[n_nationkey@0] ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name], predicate=n_name@1 = ALGERIA, pruning_predicate=CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= ALGERIA AND ALGERIA <= n_name_max@1 END, required_guarantees=[n_name in (ALGERIA)] -Query Stage #6 (1 -> 2): +Query Stage #6 (2 -> 2): ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_nationkey] @@ -167,7 +167,7 @@ ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "ps_part CoalesceBatchesExec: target_batch_size=8192 ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) -Query Stage #11 (2 -> 1): +Query Stage #11 (1 -> 1): SortPreservingMergeExec: [value@1 DESC] ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) diff --git a/testdata/expected-plans/q12.txt b/testdata/expected-plans/q12.txt deleted file mode 100644 index f2052fb..0000000 --- a/testdata/expected-plans/q12.txt +++ /dev/null @@ -1,71 +0,0 @@ -DataFusion Logical Plan -======================= - -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)]] - Projection: orders.o_orderpriority, lineitem.l_shipmode - Inner Join: orders.o_orderkey = lineitem.l_orderkey - TableScan: orders projection=[o_orderkey, o_orderpriority] - Projection: lineitem.l_orderkey, lineitem.l_shipmode - Filter: (lineitem.l_shipmode = Utf8("FOB") OR lineitem.l_shipmode = Utf8("SHIP")) AND lineitem.l_receiptdate > lineitem.l_commitdate AND lineitem.l_shipdate < lineitem.l_commitdate AND lineitem.l_receiptdate >= Date32("1995-01-01") AND lineitem.l_receiptdate < Date32("1996-01-01") - TableScan: lineitem projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], partial_filters=[lineitem.l_shipmode = Utf8("FOB") OR lineitem.l_shipmode = Utf8("SHIP"), lineitem.l_receiptdate > lineitem.l_commitdate, lineitem.l_shipdate < lineitem.l_commitdate, lineitem.l_receiptdate >= Date32("1995-01-01"), lineitem.l_receiptdate < Date32("1996-01-01")] - -DataFusion Physical Plan -======================== - -SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] - SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true] - ProjectionExec: expr=[l_shipmode@0 as 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)@1 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)@2 as low_line_count] - AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as 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)] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([l_shipmode@0], 2), input_partitions=2 - AggregateExec: mode=Partial, gby=[l_shipmode@1 as 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)] - ProjectionExec: expr=[o_orderpriority@1 as o_orderpriority, l_shipmode@0 as l_shipmode] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([l_orderkey@0], 2), input_partitions=2 - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: (l_shipmode@4 = FOB OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1995-01-01 AND l_receiptdate@3 < 1996-01-01, projection=[l_orderkey@0, l_shipmode@4] - ParquetExec: file_groups={ ... }, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], predicate=(l_shipmode@14 = FOB OR l_shipmode@14 = SHIP) AND l_receiptdate@12 > l_commitdate@11 AND l_shipdate@10 < l_commitdate@11 AND l_receiptdate@12 >= 1995-01-01 AND l_receiptdate@12 < 1996-01-01, pruning_predicate=(CASE WHEN l_shipmode_null_count@2 = l_shipmode_row_count@3 THEN false ELSE l_shipmode_min@0 <= FOB AND FOB <= l_shipmode_max@1 END OR CASE WHEN l_shipmode_null_count@2 = l_shipmode_row_count@3 THEN false ELSE l_shipmode_min@0 <= SHIP AND SHIP <= l_shipmode_max@1 END) AND CASE WHEN l_receiptdate_null_count@5 = l_receiptdate_row_count@6 THEN false ELSE l_receiptdate_max@4 >= 1995-01-01 END AND CASE WHEN l_receiptdate_null_count@5 = l_receiptdate_row_count@6 THEN false ELSE l_receiptdate_min@7 < 1996-01-01 END, required_guarantees=[l_shipmode in (SHIP, FOB)] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([o_orderkey@0], 2), input_partitions=2 - ParquetExec: file_groups={ ... }, projection=[o_orderkey, o_orderpriority] - -DataFusion Ray Distributed Plan -=========== - -Query Stage #0 (2 -> 2): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: (l_shipmode@4 = FOB OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1995-01-01 AND l_receiptdate@3 < 1996-01-01, projection=[l_orderkey@0, l_shipmode@4] - ParquetExec: file_groups={ ... }, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], predicate=(l_shipmode@14 = FOB OR l_shipmode@14 = SHIP) AND l_receiptdate@12 > l_commitdate@11 AND l_shipdate@10 < l_commitdate@11 AND l_receiptdate@12 >= 1995-01-01 AND l_receiptdate@12 < 1996-01-01, pruning_predicate=(CASE WHEN l_shipmode_null_count@2 = l_shipmode_row_count@3 THEN false ELSE l_shipmode_min@0 <= FOB AND FOB <= l_shipmode_max@1 END OR CASE WHEN l_shipmode_null_count@2 = l_shipmode_row_count@3 THEN false ELSE l_shipmode_min@0 <= SHIP AND SHIP <= l_shipmode_max@1 END) AND CASE WHEN l_receiptdate_null_count@5 = l_receiptdate_row_count@6 THEN false ELSE l_receiptdate_max@4 >= 1995-01-01 END AND CASE WHEN l_receiptdate_null_count@5 = l_receiptdate_row_count@6 THEN false ELSE l_receiptdate_min@7 < 1996-01-01 END, required_guarantees=[l_shipmode in (SHIP, FOB)] - -Query Stage #1 (2 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) - ParquetExec: file_groups={ ... }, projection=[o_orderkey, o_orderpriority] - -Query Stage #2 (2 -> 2): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 2)) - AggregateExec: mode=Partial, gby=[l_shipmode@1 as 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)] - ProjectionExec: expr=[o_orderpriority@1 as o_orderpriority, l_shipmode@0 as l_shipmode] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) - -Query Stage #3 (2 -> 2): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 2)) - SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true] - ProjectionExec: expr=[l_shipmode@0 as 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)@1 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)@2 as low_line_count] - AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as 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)] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 2)) - -Query Stage #4 (2 -> 1): -SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 2)) - diff --git a/testdata/expected-plans/q13.txt b/testdata/expected-plans/q13.txt index 691f45e..5ddc170 100644 --- a/testdata/expected-plans/q13.txt +++ b/testdata/expected-plans/q13.txt @@ -70,7 +70,7 @@ ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "c_count" CoalesceBatchesExec: target_batch_size=8192 ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "c_count", index: 0 }], 2)) -Query Stage #4 (2 -> 1): +Query Stage #4 (1 -> 1): SortPreservingMergeExec: [custdist@1 DESC,c_count@0 DESC] ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "c_count", index: 0 }], 2)) diff --git a/testdata/expected-plans/q14.txt b/testdata/expected-plans/q14.txt index 81ef8ef..8add1f2 100644 --- a/testdata/expected-plans/q14.txt +++ b/testdata/expected-plans/q14.txt @@ -33,7 +33,7 @@ ProjectionExec: expr=[100 * CAST(sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") T DataFusion Ray Distributed Plan =========== -Query Stage #0 (1 -> 2): +Query Stage #0 (2 -> 2): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[p_partkey, p_type] diff --git a/testdata/expected-plans/q16.txt b/testdata/expected-plans/q16.txt deleted file mode 100644 index 5ef333a..0000000 --- a/testdata/expected-plans/q16.txt +++ /dev/null @@ -1,113 +0,0 @@ -DataFusion Logical Plan -======================= - -Sort: supplier_cnt DESC NULLS FIRST, part.p_brand ASC NULLS LAST, part.p_type ASC NULLS LAST, part.p_size ASC NULLS LAST - Projection: part.p_brand, part.p_type, part.p_size, count(alias1) AS supplier_cnt - Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size]], aggr=[[count(alias1)]] - Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size, partsupp.ps_suppkey AS alias1]], aggr=[[]] - LeftAnti Join: partsupp.ps_suppkey = __correlated_sq_1.s_suppkey - 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#14") AND part.p_type NOT LIKE Utf8("SMALL PLATED%") AND part.p_size IN ([Int32(14), Int32(6), Int32(5), Int32(31), Int32(49), Int32(15), Int32(41), Int32(47)]) - TableScan: part projection=[p_partkey, p_brand, p_type, p_size], partial_filters=[part.p_brand != Utf8("Brand#14"), part.p_type NOT LIKE Utf8("SMALL PLATED%"), part.p_size IN ([Int32(14), Int32(6), Int32(5), Int32(31), Int32(49), Int32(15), Int32(41), Int32(47)])] - SubqueryAlias: __correlated_sq_1 - Projection: supplier.s_suppkey - Filter: supplier.s_comment LIKE Utf8("%Customer%Complaints%") - TableScan: supplier projection=[s_suppkey, s_comment], partial_filters=[supplier.s_comment LIKE Utf8("%Customer%Complaints%")] - -DataFusion Physical Plan -======================== - -SortPreservingMergeExec: [supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST] - SortExec: expr=[supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST], preserve_partitioning=[true] - ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, count(alias1)@3 as supplier_cnt] - AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 2), input_partitions=2 - AggregateExec: mode=Partial, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] - AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, alias1@3 as alias1], aggr=[] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 2), input_partitions=2 - AggregateExec: mode=Partial, gby=[p_brand@1 as p_brand, p_type@2 as p_type, p_size@3 as p_size, ps_suppkey@0 as alias1], aggr=[] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=RightAnti, on=[(s_suppkey@0, ps_suppkey@0)] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([s_suppkey@0], 2), input_partitions=2 - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] - RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 - ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_comment], predicate=s_comment@6 LIKE %Customer%Complaints% - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([ps_suppkey@0], 2), input_partitions=2 - ProjectionExec: expr=[ps_suppkey@3 as ps_suppkey, p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_brand@1, p_type@2, p_size@3, ps_suppkey@5] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([p_partkey@0], 2), input_partitions=2 - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: p_brand@1 != Brand#14 AND p_type@2 NOT LIKE SMALL PLATED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(14) }, Literal { value: Int32(6) }, Literal { value: Int32(5) }, Literal { value: Int32(31) }, Literal { value: Int32(49) }, Literal { value: Int32(15) }, Literal { value: Int32(41) }, Literal { value: Int32(47) }]) - RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 - ParquetExec: file_groups={ ... }]), pruning_predicate=CASE WHEN p_brand_null_count@2 = p_brand_row_count@3 THEN false ELSE p_brand_min@0 != Brand#14 OR Brand#14 != p_brand_max@1 END AND (CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 14 AND 14 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 6 AND 6 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 5 AND 5 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 31 AND 31 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 49 AND 49 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 15 AND 15 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 41 AND 41 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 47 AND 47 <= p_size_max@5 END), required_guarantees=[p_brand not in (Brand#14), p_size in (5, 41, 49, 15, 6, 31, 47, 14)] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([ps_partkey@0], 2), input_partitions=2 - ParquetExec: file_groups={ ... }, projection=[ps_partkey, ps_suppkey] - -DataFusion Ray Distributed Plan -=========== - -Query Stage #0 (1 -> 2): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] - ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_comment], predicate=s_comment@6 LIKE %Customer%Complaints% - -Query Stage #1 (1 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: p_brand@1 != Brand#14 AND p_type@2 NOT LIKE SMALL PLATED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(14) }, Literal { value: Int32(6) }, Literal { value: Int32(5) }, Literal { value: Int32(31) }, Literal { value: Int32(49) }, Literal { value: Int32(15) }, Literal { value: Int32(41) }, Literal { value: Int32(47) }]) - ParquetExec: file_groups={ ... }]), pruning_predicate=CASE WHEN p_brand_null_count@2 = p_brand_row_count@3 THEN false ELSE p_brand_min@0 != Brand#14 OR Brand#14 != p_brand_max@1 END AND (CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 14 AND 14 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 6 AND 6 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 5 AND 5 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 31 AND 31 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 49 AND 49 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 15 AND 15 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 41 AND 41 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 47 AND 47 <= p_size_max@5 END), required_guarantees=[p_brand not in (Brand#14), p_size in (5, 41, 49, 15, 6, 31, 47, 14)] - -Query Stage #2 (2 -> 2): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) - ParquetExec: file_groups={ ... }, projection=[ps_partkey, ps_suppkey] - -Query Stage #3 (2 -> 2): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 2)) - ProjectionExec: expr=[ps_suppkey@3 as ps_suppkey, p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_brand@1, p_type@2, p_size@3, ps_suppkey@5] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) - -Query Stage #4 (2 -> 2): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "p_brand", index: 0 }, Column { name: "p_type", index: 1 }, Column { name: "p_size", index: 2 }, Column { name: "alias1", index: 3 }], 2)) - AggregateExec: mode=Partial, gby=[p_brand@1 as p_brand, p_type@2 as p_type, p_size@3 as p_size, ps_suppkey@0 as alias1], aggr=[] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=RightAnti, on=[(s_suppkey@0, ps_suppkey@0)] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 2)) - -Query Stage #5 (2 -> 2): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "p_brand", index: 0 }, Column { name: "p_type", index: 1 }, Column { name: "p_size", index: 2 }], 2)) - AggregateExec: mode=Partial, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] - AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, alias1@3 as alias1], aggr=[] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "p_brand", index: 0 }, Column { name: "p_type", index: 1 }, Column { name: "p_size", index: 2 }, Column { name: "alias1", index: 3 }], 2)) - -Query Stage #6 (2 -> 2): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "p_brand", index: 0 }, Column { name: "p_type", index: 1 }, Column { name: "p_size", index: 2 }], 2)) - SortExec: expr=[supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST], preserve_partitioning=[true] - ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, count(alias1)@3 as supplier_cnt] - AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "p_brand", index: 0 }, Column { name: "p_type", index: 1 }, Column { name: "p_size", index: 2 }], 2)) - -Query Stage #7 (2 -> 1): -SortPreservingMergeExec: [supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST] - ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "p_brand", index: 0 }, Column { name: "p_type", index: 1 }, Column { name: "p_size", index: 2 }], 2)) - diff --git a/testdata/expected-plans/q17.txt b/testdata/expected-plans/q17.txt index 454f0ad..d86d08c 100644 --- a/testdata/expected-plans/q17.txt +++ b/testdata/expected-plans/q17.txt @@ -47,7 +47,7 @@ ProjectionExec: expr=[CAST(sum(lineitem.l_extendedprice)@0 AS Float64) / 7 as av DataFusion Ray Distributed Plan =========== -Query Stage #0 (1 -> 2): +Query Stage #0 (2 -> 2): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: p_brand@1 = Brand#42 AND p_container@2 = LG BAG, projection=[p_partkey@0] diff --git a/testdata/expected-plans/q18.txt b/testdata/expected-plans/q18.txt index 0696af7..468884c 100644 --- a/testdata/expected-plans/q18.txt +++ b/testdata/expected-plans/q18.txt @@ -104,7 +104,7 @@ ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "c_name", CoalesceBatchesExec: target_batch_size=8192 ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "c_name", index: 0 }, Column { name: "c_custkey", index: 1 }, Column { name: "o_orderkey", index: 2 }, Column { name: "o_orderdate", index: 3 }, Column { name: "o_totalprice", index: 4 }], 2)) -Query Stage #7 (2 -> 1): +Query Stage #7 (1 -> 1): SortPreservingMergeExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST], fetch=100 ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "c_name", index: 0 }, Column { name: "c_custkey", index: 1 }, Column { name: "o_orderkey", index: 2 }, Column { name: "o_orderdate", index: 3 }, Column { name: "o_totalprice", index: 4 }], 2)) diff --git a/testdata/expected-plans/q19.txt b/testdata/expected-plans/q19.txt deleted file mode 100644 index c98f39e..0000000 --- a/testdata/expected-plans/q19.txt +++ /dev/null @@ -1,65 +0,0 @@ -DataFusion Logical Plan -======================= - -Projection: sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue - Aggregate: groupBy=[[]], aggr=[[sum(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] - Projection: lineitem.l_extendedprice, lineitem.l_discount - Inner Join: lineitem.l_partkey = part.p_partkey Filter: part.p_brand = Utf8("Brand#21") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND lineitem.l_quantity >= Decimal128(Some(800),11,2) AND lineitem.l_quantity <= Decimal128(Some(1800),11,2) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#13") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND lineitem.l_quantity >= Decimal128(Some(2000),11,2) AND lineitem.l_quantity <= Decimal128(Some(3000),11,2) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#52") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND lineitem.l_quantity >= Decimal128(Some(3000),11,2) AND lineitem.l_quantity <= Decimal128(Some(4000),11,2) AND part.p_size <= Int32(15) - Projection: lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount - Filter: (lineitem.l_quantity >= Decimal128(Some(800),11,2) AND lineitem.l_quantity <= Decimal128(Some(1800),11,2) OR lineitem.l_quantity >= Decimal128(Some(2000),11,2) AND lineitem.l_quantity <= Decimal128(Some(3000),11,2) OR lineitem.l_quantity >= Decimal128(Some(3000),11,2) AND lineitem.l_quantity <= Decimal128(Some(4000),11,2)) AND (lineitem.l_shipmode = Utf8("AIR") OR lineitem.l_shipmode = Utf8("AIR REG")) AND lineitem.l_shipinstruct = Utf8("DELIVER IN PERSON") - TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], partial_filters=[lineitem.l_shipmode = Utf8("AIR") OR lineitem.l_shipmode = Utf8("AIR REG"), lineitem.l_shipinstruct = Utf8("DELIVER IN PERSON"), lineitem.l_quantity >= Decimal128(Some(800),11,2) AND lineitem.l_quantity <= Decimal128(Some(1800),11,2) OR lineitem.l_quantity >= Decimal128(Some(2000),11,2) AND lineitem.l_quantity <= Decimal128(Some(3000),11,2) OR lineitem.l_quantity >= Decimal128(Some(3000),11,2) AND lineitem.l_quantity <= Decimal128(Some(4000),11,2)] - Filter: (part.p_brand = Utf8("Brand#21") 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#13") 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#52") 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], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8("Brand#21") 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#13") 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#52") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND part.p_size <= Int32(15)] - -DataFusion Physical Plan -======================== - -ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue] - AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - CoalescePartitionsExec - AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], filter=p_brand@1 = Brand#21 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND l_quantity@0 >= Some(800),11,2 AND l_quantity@0 <= Some(1800),11,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#13 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND l_quantity@0 >= Some(2000),11,2 AND l_quantity@0 <= Some(3000),11,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#52 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND l_quantity@0 >= Some(3000),11,2 AND l_quantity@0 <= Some(4000),11,2 AND p_size@2 <= 15, projection=[l_extendedprice@6, l_discount@7] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([p_partkey@0], 2), input_partitions=2 - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: (p_brand@1 = Brand#21 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#13 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#52 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND p_size@2 <= 15) AND p_size@2 >= 1 - RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 - ParquetExec: file_groups={ ... }]) AND p_size@5 <= 15), pruning_predicate=CASE WHEN p_size_null_count@1 = p_size_row_count@2 THEN false ELSE p_size_max@0 >= 1 END AND (CASE WHEN p_brand_null_count@5 = p_brand_row_count@6 THEN false ELSE p_brand_min@3 <= Brand#21 AND Brand#21 <= p_brand_max@4 END AND (CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= SM CASE AND SM CASE <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= SM BOX AND SM BOX <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= SM PACK AND SM PACK <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= SM PKG AND SM PKG <= p_container_max@8 END) AND CASE WHEN p_size_null_count@1 = p_size_row_count@2 THEN false ELSE p_size_min@11 <= 5 END OR CASE WHEN p_brand_null_count@5 = p_brand_row_count@6 THEN false ELSE p_brand_min@3 <= Brand#13 AND Brand#13 <= p_brand_max@4 END AND (CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= MED BAG AND MED BAG <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= MED BOX AND MED BOX <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= MED PKG AND MED PKG <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= MED PACK AND MED PACK <= p_container_max@8 END) AND CASE WHEN p_size_null_count@1 = p_size_row_count@2 THEN false ELSE p_size_min@11 <= 10 END OR CASE WHEN p_brand_null_count@5 = p_brand_row_count@6 THEN false ELSE p_brand_min@3 <= Brand#52 AND Brand#52 <= p_brand_max@4 END AND (CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= LG CASE AND LG CASE <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= LG BOX AND LG BOX <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= LG PACK AND LG PACK <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= LG PKG AND LG PKG <= p_container_max@8 END) AND CASE WHEN p_size_null_count@1 = p_size_row_count@2 THEN false ELSE p_size_min@11 <= 15 END), required_guarantees=[] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([l_partkey@0], 2), input_partitions=2 - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: (l_quantity@1 >= Some(800),11,2 AND l_quantity@1 <= Some(1800),11,2 OR l_quantity@1 >= Some(2000),11,2 AND l_quantity@1 <= Some(3000),11,2 OR l_quantity@1 >= Some(3000),11,2 AND l_quantity@1 <= Some(4000),11,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON, projection=[l_partkey@0, l_quantity@1, l_extendedprice@2, l_discount@3] - ParquetExec: file_groups={ ... }, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], predicate=(l_shipmode@14 = AIR OR l_shipmode@14 = AIR REG) AND l_shipinstruct@13 = DELIVER IN PERSON AND (l_quantity@4 >= Some(800),11,2 AND l_quantity@4 <= Some(1800),11,2 OR l_quantity@4 >= Some(2000),11,2 AND l_quantity@4 <= Some(3000),11,2 OR l_quantity@4 >= Some(3000),11,2 AND l_quantity@4 <= Some(4000),11,2), pruning_predicate=(CASE WHEN l_shipmode_null_count@2 = l_shipmode_row_count@3 THEN false ELSE l_shipmode_min@0 <= AIR AND AIR <= l_shipmode_max@1 END OR CASE WHEN l_shipmode_null_count@2 = l_shipmode_row_count@3 THEN false ELSE l_shipmode_min@0 <= AIR REG AND AIR REG <= l_shipmode_max@1 END) AND CASE WHEN l_shipinstruct_null_count@6 = l_shipinstruct_row_count@7 THEN false ELSE l_shipinstruct_min@4 <= DELIVER IN PERSON AND DELIVER IN PERSON <= l_shipinstruct_max@5 END AND (CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_max@8 >= Some(800),11,2 END AND CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_min@11 <= Some(1800),11,2 END OR CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_max@8 >= Some(2000),11,2 END AND CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_min@11 <= Some(3000),11,2 END OR CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_max@8 >= Some(3000),11,2 END AND CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_min@11 <= Some(4000),11,2 END), required_guarantees=[l_shipmode in (AIR REG, AIR), l_shipinstruct in (DELIVER IN PERSON)] - -DataFusion Ray Distributed Plan -=========== - -Query Stage #0 (1 -> 2): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: (p_brand@1 = Brand#21 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#13 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#52 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND p_size@2 <= 15) AND p_size@2 >= 1 - ParquetExec: file_groups={ ... }]) AND p_size@5 <= 15), pruning_predicate=CASE WHEN p_size_null_count@1 = p_size_row_count@2 THEN false ELSE p_size_max@0 >= 1 END AND (CASE WHEN p_brand_null_count@5 = p_brand_row_count@6 THEN false ELSE p_brand_min@3 <= Brand#21 AND Brand#21 <= p_brand_max@4 END AND (CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= SM CASE AND SM CASE <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= SM BOX AND SM BOX <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= SM PACK AND SM PACK <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= SM PKG AND SM PKG <= p_container_max@8 END) AND CASE WHEN p_size_null_count@1 = p_size_row_count@2 THEN false ELSE p_size_min@11 <= 5 END OR CASE WHEN p_brand_null_count@5 = p_brand_row_count@6 THEN false ELSE p_brand_min@3 <= Brand#13 AND Brand#13 <= p_brand_max@4 END AND (CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= MED BAG AND MED BAG <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= MED BOX AND MED BOX <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= MED PKG AND MED PKG <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= MED PACK AND MED PACK <= p_container_max@8 END) AND CASE WHEN p_size_null_count@1 = p_size_row_count@2 THEN false ELSE p_size_min@11 <= 10 END OR CASE WHEN p_brand_null_count@5 = p_brand_row_count@6 THEN false ELSE p_brand_min@3 <= Brand#52 AND Brand#52 <= p_brand_max@4 END AND (CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= LG CASE AND LG CASE <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= LG BOX AND LG BOX <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= LG PACK AND LG PACK <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= LG PKG AND LG PKG <= p_container_max@8 END) AND CASE WHEN p_size_null_count@1 = p_size_row_count@2 THEN false ELSE p_size_min@11 <= 15 END), required_guarantees=[] - -Query Stage #1 (2 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 2)) - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: (l_quantity@1 >= Some(800),11,2 AND l_quantity@1 <= Some(1800),11,2 OR l_quantity@1 >= Some(2000),11,2 AND l_quantity@1 <= Some(3000),11,2 OR l_quantity@1 >= Some(3000),11,2 AND l_quantity@1 <= Some(4000),11,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON, projection=[l_partkey@0, l_quantity@1, l_extendedprice@2, l_discount@3] - ParquetExec: file_groups={ ... }, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], predicate=(l_shipmode@14 = AIR OR l_shipmode@14 = AIR REG) AND l_shipinstruct@13 = DELIVER IN PERSON AND (l_quantity@4 >= Some(800),11,2 AND l_quantity@4 <= Some(1800),11,2 OR l_quantity@4 >= Some(2000),11,2 AND l_quantity@4 <= Some(3000),11,2 OR l_quantity@4 >= Some(3000),11,2 AND l_quantity@4 <= Some(4000),11,2), pruning_predicate=(CASE WHEN l_shipmode_null_count@2 = l_shipmode_row_count@3 THEN false ELSE l_shipmode_min@0 <= AIR AND AIR <= l_shipmode_max@1 END OR CASE WHEN l_shipmode_null_count@2 = l_shipmode_row_count@3 THEN false ELSE l_shipmode_min@0 <= AIR REG AND AIR REG <= l_shipmode_max@1 END) AND CASE WHEN l_shipinstruct_null_count@6 = l_shipinstruct_row_count@7 THEN false ELSE l_shipinstruct_min@4 <= DELIVER IN PERSON AND DELIVER IN PERSON <= l_shipinstruct_max@5 END AND (CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_max@8 >= Some(800),11,2 END AND CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_min@11 <= Some(1800),11,2 END OR CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_max@8 >= Some(2000),11,2 END AND CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_min@11 <= Some(3000),11,2 END OR CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_max@8 >= Some(3000),11,2 END AND CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_min@11 <= Some(4000),11,2 END), required_guarantees=[l_shipmode in (AIR REG, AIR), l_shipinstruct in (DELIVER IN PERSON)] - -Query Stage #2 (2 -> 1): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([], 2)) - AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], filter=p_brand@1 = Brand#21 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND l_quantity@0 >= Some(800),11,2 AND l_quantity@0 <= Some(1800),11,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#13 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND l_quantity@0 >= Some(2000),11,2 AND l_quantity@0 <= Some(3000),11,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#52 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND l_quantity@0 >= Some(3000),11,2 AND l_quantity@0 <= Some(4000),11,2 AND p_size@2 <= 15, projection=[l_extendedprice@6, l_discount@7] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 2)) - -Query Stage #3 (1 -> 1): -ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue] - AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - CoalescePartitionsExec - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([], 2)) - diff --git a/testdata/expected-plans/q2.txt b/testdata/expected-plans/q2.txt index cb67479..3ac7ebd 100644 --- a/testdata/expected-plans/q2.txt +++ b/testdata/expected-plans/q2.txt @@ -124,21 +124,21 @@ SortPreservingMergeExec: [s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC DataFusion Ray Distributed Plan =========== -Query Stage #0 (1 -> 2): +Query Stage #0 (2 -> 2): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] ParquetExec: file_groups={ ... }, projection=[r_regionkey, r_name], predicate=r_name@1 = ASIA, pruning_predicate=CASE WHEN r_name_null_count@2 = r_name_row_count@3 THEN false ELSE r_name_min@0 <= ASIA AND ASIA <= r_name_max@1 END, required_guarantees=[r_name in (ASIA)] -Query Stage #1 (1 -> 2): +Query Stage #1 (2 -> 2): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name, n_regionkey] -Query Stage #2 (1 -> 2): +Query Stage #2 (2 -> 2): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] -Query Stage #3 (1 -> 2): +Query Stage #3 (2 -> 2): ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: p_size@3 = 48 AND p_type@2 LIKE %TIN, projection=[p_partkey@0, p_mfgr@1] @@ -186,17 +186,17 @@ ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "p_partke CoalesceBatchesExec: target_batch_size=8192 ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "n_regionkey", index: 9 }], 2)) -Query Stage #9 (1 -> 2): +Query Stage #9 (2 -> 2): ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] ParquetExec: file_groups={ ... }, projection=[r_regionkey, r_name], predicate=r_name@1 = ASIA, pruning_predicate=CASE WHEN r_name_null_count@2 = r_name_row_count@3 THEN false ELSE r_name_min@0 <= ASIA AND ASIA <= r_name_max@1 END, required_guarantees=[r_name in (ASIA)] -Query Stage #10 (1 -> 2): +Query Stage #10 (2 -> 2): ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_regionkey] -Query Stage #11 (1 -> 2): +Query Stage #11 (2 -> 2): ShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_nationkey] @@ -252,7 +252,7 @@ ShuffleWriterExec(stage_id=17, output_partitioning=Hash([Column { name: "p_partk CoalesceBatchesExec: target_batch_size=8192 ShuffleReaderExec(stage_id=16, input_partitioning=Hash([Column { name: "ps_partkey", index: 1 }, Column { name: "min(partsupp.ps_supplycost)", index: 0 }], 2)) -Query Stage #18 (2 -> 1): +Query Stage #18 (1 -> 1): SortPreservingMergeExec: [s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST], fetch=100 ShuffleReaderExec(stage_id=17, input_partitioning=Hash([Column { name: "p_partkey", index: 3 }], 2)) diff --git a/testdata/expected-plans/q20.txt b/testdata/expected-plans/q20.txt index 5473093..3ab727e 100644 --- a/testdata/expected-plans/q20.txt +++ b/testdata/expected-plans/q20.txt @@ -76,13 +76,13 @@ SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] DataFusion Ray Distributed Plan =========== -Query Stage #0 (1 -> 2): +Query Stage #0 (2 -> 2): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: n_name@1 = KENYA, projection=[n_nationkey@0] ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name], predicate=n_name@1 = KENYA, pruning_predicate=CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= KENYA AND KENYA <= n_name_max@1 END, required_guarantees=[n_name in (KENYA)] -Query Stage #1 (1 -> 2): +Query Stage #1 (2 -> 2): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 2)) ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_name, s_address, s_nationkey] @@ -95,7 +95,7 @@ ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "s_suppke CoalesceBatchesExec: target_batch_size=8192 ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 2)) -Query Stage #3 (1 -> 2): +Query Stage #3 (2 -> 2): ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: p_name@1 LIKE blanched%, projection=[p_partkey@0] @@ -142,7 +142,7 @@ ShuffleWriterExec(stage_id=8, output_partitioning=Hash([], 2)) CoalesceBatchesExec: target_batch_size=8192 ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 2)) -Query Stage #9 (2 -> 1): +Query Stage #9 (1 -> 1): SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] ShuffleReaderExec(stage_id=8, input_partitioning=Hash([], 2)) diff --git a/testdata/expected-plans/q21.txt b/testdata/expected-plans/q21.txt index dbd5e97..52f1862 100644 --- a/testdata/expected-plans/q21.txt +++ b/testdata/expected-plans/q21.txt @@ -91,7 +91,7 @@ SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST], fetch=100 DataFusion Ray Distributed Plan =========== -Query Stage #0 (1 -> 2): +Query Stage #0 (2 -> 2): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: n_name@1 = ARGENTINA, projection=[n_nationkey@0] @@ -103,7 +103,7 @@ ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_orderk FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] ParquetExec: file_groups={ ... }, projection=[o_orderkey, o_orderstatus], predicate=o_orderstatus@2 = F, pruning_predicate=CASE WHEN o_orderstatus_null_count@2 = o_orderstatus_row_count@3 THEN false ELSE o_orderstatus_min@0 <= F AND F <= o_orderstatus_max@1 END, required_guarantees=[o_orderstatus in (F)] -Query Stage #2 (1 -> 2): +Query Stage #2 (2 -> 2): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_name, s_nationkey] @@ -172,7 +172,7 @@ ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "s_name" CoalesceBatchesExec: target_batch_size=8192 ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "s_name", index: 0 }], 2)) -Query Stage #11 (2 -> 1): +Query Stage #11 (1 -> 1): SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST], fetch=100 ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "s_name", index: 0 }], 2)) diff --git a/testdata/expected-plans/q22.txt b/testdata/expected-plans/q22.txt index d46d5d5..1e3c4ad 100644 --- a/testdata/expected-plans/q22.txt +++ b/testdata/expected-plans/q22.txt @@ -91,7 +91,7 @@ ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "cntrycod CoalesceBatchesExec: target_batch_size=8192 ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "cntrycode", index: 0 }], 2)) -Query Stage #5 (2 -> 1): +Query Stage #5 (1 -> 1): SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "cntrycode", index: 0 }], 2)) diff --git a/testdata/expected-plans/q3.txt b/testdata/expected-plans/q3.txt index 6fd8791..8f4e0c2 100644 --- a/testdata/expected-plans/q3.txt +++ b/testdata/expected-plans/q3.txt @@ -97,7 +97,7 @@ ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "l_orderk CoalesceBatchesExec: target_batch_size=8192 ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 1 }, Column { name: "o_shippriority", index: 2 }], 2)) -Query Stage #6 (2 -> 1): +Query Stage #6 (1 -> 1): SortPreservingMergeExec: [revenue@1 DESC,o_orderdate@2 ASC NULLS LAST], fetch=10 ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 2 }, Column { name: "o_shippriority", index: 3 }], 2)) diff --git a/testdata/expected-plans/q4.txt b/testdata/expected-plans/q4.txt index 20460e4..2504483 100644 --- a/testdata/expected-plans/q4.txt +++ b/testdata/expected-plans/q4.txt @@ -70,7 +70,7 @@ ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "o_orderp CoalesceBatchesExec: target_batch_size=8192 ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 2)) -Query Stage #4 (2 -> 1): +Query Stage #4 (1 -> 1): SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 2)) diff --git a/testdata/expected-plans/q5.txt b/testdata/expected-plans/q5.txt index 5351e06..25c047b 100644 --- a/testdata/expected-plans/q5.txt +++ b/testdata/expected-plans/q5.txt @@ -83,17 +83,17 @@ SortPreservingMergeExec: [revenue@1 DESC] DataFusion Ray Distributed Plan =========== -Query Stage #0 (1 -> 2): +Query Stage #0 (2 -> 2): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: r_name@1 = AFRICA, projection=[r_regionkey@0] ParquetExec: file_groups={ ... }, projection=[r_regionkey, r_name], predicate=r_name@1 = AFRICA, pruning_predicate=CASE WHEN r_name_null_count@2 = r_name_row_count@3 THEN false ELSE r_name_min@0 <= AFRICA AND AFRICA <= r_name_max@1 END, required_guarantees=[r_name in (AFRICA)] -Query Stage #1 (1 -> 2): +Query Stage #1 (2 -> 2): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name, n_regionkey] -Query Stage #2 (1 -> 2): +Query Stage #2 (2 -> 2): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }, Column { name: "s_nationkey", index: 1 }], 2)) ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_nationkey] @@ -167,7 +167,7 @@ ShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "n_name" CoalesceBatchesExec: target_batch_size=8192 ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "n_name", index: 0 }], 2)) -Query Stage #12 (2 -> 1): +Query Stage #12 (1 -> 1): SortPreservingMergeExec: [revenue@1 DESC] ShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "n_name", index: 0 }], 2)) diff --git a/testdata/expected-plans/q7.txt b/testdata/expected-plans/q7.txt deleted file mode 100644 index b9e261a..0000000 --- a/testdata/expected-plans/q7.txt +++ /dev/null @@ -1,182 +0,0 @@ -DataFusion Logical Plan -======================= - -Sort: shipping.supp_nation ASC NULLS LAST, shipping.cust_nation ASC NULLS LAST, shipping.l_year ASC NULLS LAST - Projection: shipping.supp_nation, shipping.cust_nation, shipping.l_year, sum(shipping.volume) AS revenue - Aggregate: groupBy=[[shipping.supp_nation, shipping.cust_nation, shipping.l_year]], aggr=[[sum(shipping.volume)]] - SubqueryAlias: shipping - Projection: n1.n_name AS supp_nation, n2.n_name AS cust_nation, date_part(Utf8("YEAR"), lineitem.l_shipdate) AS l_year, lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS volume - Inner Join: customer.c_nationkey = n2.n_nationkey Filter: n1.n_name = Utf8("GERMANY") AND n2.n_name = Utf8("IRAQ") OR n1.n_name = Utf8("IRAQ") AND n2.n_name = Utf8("GERMANY") - 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("1995-01-01") AND lineitem.l_shipdate <= Date32("1996-12-31") - TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("1995-01-01"), lineitem.l_shipdate <= Date32("1996-12-31")] - TableScan: orders projection=[o_orderkey, o_custkey] - TableScan: customer projection=[c_custkey, c_nationkey] - SubqueryAlias: n1 - Filter: nation.n_name = Utf8("GERMANY") OR nation.n_name = Utf8("IRAQ") - TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("GERMANY") OR nation.n_name = Utf8("IRAQ")] - SubqueryAlias: n2 - Filter: nation.n_name = Utf8("IRAQ") OR nation.n_name = Utf8("GERMANY") - TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("IRAQ") OR nation.n_name = Utf8("GERMANY")] - -DataFusion Physical Plan -======================== - -SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST] - SortExec: expr=[supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST], preserve_partitioning=[true] - ProjectionExec: expr=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year, sum(shipping.volume)@3 as revenue] - AggregateExec: mode=FinalPartitioned, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 2), input_partitions=2 - AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] - ProjectionExec: expr=[n_name@4 as supp_nation, n_name@0 as cust_nation, date_part(YEAR, l_shipdate@3) as l_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as volume] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_nationkey@0, c_nationkey@3)], filter=n_name@0 = GERMANY AND n_name@1 = IRAQ OR n_name@0 = IRAQ AND n_name@1 = GERMANY, projection=[n_name@1, l_extendedprice@2, l_discount@3, l_shipdate@4, n_name@6] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([n_nationkey@0], 2), input_partitions=2 - RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: n_name@1 = IRAQ OR n_name@1 = GERMANY - ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name], predicate=n_name@1 = IRAQ OR n_name@1 = GERMANY, pruning_predicate=CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= IRAQ AND IRAQ <= n_name_max@1 END OR CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1 END, required_guarantees=[n_name in (GERMANY, IRAQ)] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([c_nationkey@3], 2), input_partitions=2 - ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@4 as c_nationkey, n_name@0 as n_name] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_nationkey@0, s_nationkey@0)], projection=[n_name@1, l_extendedprice@3, l_discount@4, l_shipdate@5, c_nationkey@6] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([n_nationkey@0], 2), input_partitions=2 - RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: n_name@1 = GERMANY OR n_name@1 = IRAQ - ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name], predicate=n_name@1 = GERMANY OR n_name@1 = IRAQ, pruning_predicate=CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1 END OR CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= IRAQ AND IRAQ <= n_name_max@1 END, required_guarantees=[n_name in (GERMANY, IRAQ)] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([s_nationkey@0], 2), input_partitions=2 - ProjectionExec: expr=[s_nationkey@1 as s_nationkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, l_shipdate@4 as l_shipdate, c_nationkey@0 as c_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@4)], projection=[c_nationkey@1, s_nationkey@2, l_extendedprice@3, l_discount@4, l_shipdate@5] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([c_custkey@0], 2), input_partitions=2 - ParquetExec: file_groups={ ... }, projection=[c_custkey, c_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([o_custkey@4], 2), input_partitions=2 - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([l_orderkey@1], 2), input_partitions=2 - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([s_suppkey@0], 2), input_partitions=2 - RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 - ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([l_suppkey@1], 2), input_partitions=2 - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 - ParquetExec: file_groups={ ... }, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], predicate=l_shipdate@10 >= 1995-01-01 AND l_shipdate@10 <= 1996-12-31, pruning_predicate=CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_max@0 >= 1995-01-01 END AND CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_min@3 <= 1996-12-31 END, required_guarantees=[] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([o_orderkey@0], 2), input_partitions=2 - ParquetExec: file_groups={ ... }, projection=[o_orderkey, o_custkey] - -DataFusion Ray Distributed Plan -=========== - -Query Stage #0 (1 -> 2): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: n_name@1 = IRAQ OR n_name@1 = GERMANY - ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name], predicate=n_name@1 = IRAQ OR n_name@1 = GERMANY, pruning_predicate=CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= IRAQ AND IRAQ <= n_name_max@1 END OR CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1 END, required_guarantees=[n_name in (GERMANY, IRAQ)] - -Query Stage #1 (1 -> 2): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: n_name@1 = GERMANY OR n_name@1 = IRAQ - ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name], predicate=n_name@1 = GERMANY OR n_name@1 = IRAQ, pruning_predicate=CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1 END OR CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= IRAQ AND IRAQ <= n_name_max@1 END, required_guarantees=[n_name in (GERMANY, IRAQ)] - -Query Stage #2 (2 -> 2): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) - ParquetExec: file_groups={ ... }, projection=[c_custkey, c_nationkey] - -Query Stage #3 (1 -> 2): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) - ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_nationkey] - -Query Stage #4 (2 -> 2): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 2)) - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 - ParquetExec: file_groups={ ... }, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], predicate=l_shipdate@10 >= 1995-01-01 AND l_shipdate@10 <= 1996-12-31, pruning_predicate=CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_max@0 >= 1995-01-01 END AND CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_min@3 <= 1996-12-31 END, required_guarantees=[] - -Query Stage #5 (2 -> 2): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "l_orderkey", index: 1 }], 2)) - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 2)) - -Query Stage #6 (2 -> 2): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) - ParquetExec: file_groups={ ... }, projection=[o_orderkey, o_custkey] - -Query Stage #7 (2 -> 2): -ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "o_custkey", index: 4 }], 2)) - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "l_orderkey", index: 1 }], 2)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) - -Query Stage #8 (2 -> 2): -ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "s_nationkey", index: 0 }], 2)) - ProjectionExec: expr=[s_nationkey@1 as s_nationkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, l_shipdate@4 as l_shipdate, c_nationkey@0 as c_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@4)], projection=[c_nationkey@1, s_nationkey@2, l_extendedprice@3, l_discount@4, l_shipdate@5] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "o_custkey", index: 4 }], 2)) - -Query Stage #9 (2 -> 2): -ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "c_nationkey", index: 3 }], 2)) - ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@4 as c_nationkey, n_name@0 as n_name] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_nationkey@0, s_nationkey@0)], projection=[n_name@1, l_extendedprice@3, l_discount@4, l_shipdate@5, c_nationkey@6] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "s_nationkey", index: 0 }], 2)) - -Query Stage #10 (2 -> 2): -ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "supp_nation", index: 0 }, Column { name: "cust_nation", index: 1 }, Column { name: "l_year", index: 2 }], 2)) - AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] - ProjectionExec: expr=[n_name@4 as supp_nation, n_name@0 as cust_nation, date_part(YEAR, l_shipdate@3) as l_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as volume] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_nationkey@0, c_nationkey@3)], filter=n_name@0 = GERMANY AND n_name@1 = IRAQ OR n_name@0 = IRAQ AND n_name@1 = GERMANY, projection=[n_name@1, l_extendedprice@2, l_discount@3, l_shipdate@4, n_name@6] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "c_nationkey", index: 3 }], 2)) - -Query Stage #11 (2 -> 2): -ShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "supp_nation", index: 0 }, Column { name: "cust_nation", index: 1 }, Column { name: "l_year", index: 2 }], 2)) - SortExec: expr=[supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST], preserve_partitioning=[true] - ProjectionExec: expr=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year, sum(shipping.volume)@3 as revenue] - AggregateExec: mode=FinalPartitioned, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "supp_nation", index: 0 }, Column { name: "cust_nation", index: 1 }, Column { name: "l_year", index: 2 }], 2)) - -Query Stage #12 (2 -> 1): -SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST] - ShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "supp_nation", index: 0 }, Column { name: "cust_nation", index: 1 }, Column { name: "l_year", index: 2 }], 2)) - diff --git a/testdata/expected-plans/q8.txt b/testdata/expected-plans/q8.txt index f2333a4..d016d84 100644 --- a/testdata/expected-plans/q8.txt +++ b/testdata/expected-plans/q8.txt @@ -114,17 +114,17 @@ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] DataFusion Ray Distributed Plan =========== -Query Stage #0 (1 -> 2): +Query Stage #0 (2 -> 2): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: r_name@1 = MIDDLE EAST, projection=[r_regionkey@0] ParquetExec: file_groups={ ... }, projection=[r_regionkey, r_name], predicate=r_name@1 = MIDDLE EAST, pruning_predicate=CASE WHEN r_name_null_count@2 = r_name_row_count@3 THEN false ELSE r_name_min@0 <= MIDDLE EAST AND MIDDLE EAST <= r_name_max@1 END, required_guarantees=[r_name in (MIDDLE EAST)] -Query Stage #1 (1 -> 2): +Query Stage #1 (2 -> 2): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name] -Query Stage #2 (1 -> 2): +Query Stage #2 (2 -> 2): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_regionkey] @@ -138,11 +138,11 @@ ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "o_orderk FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 ParquetExec: file_groups={ ... }, projection=[o_orderkey, o_custkey, o_orderdate], predicate=o_orderdate@4 >= 1995-01-01 AND o_orderdate@4 <= 1996-12-31, pruning_predicate=CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_max@0 >= 1995-01-01 END AND CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_min@3 <= 1996-12-31 END, required_guarantees=[] -Query Stage #5 (1 -> 2): +Query Stage #5 (2 -> 2): ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_nationkey] -Query Stage #6 (1 -> 2): +Query Stage #6 (2 -> 2): ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: p_type@1 = LARGE PLATED STEEL, projection=[p_partkey@0] @@ -230,7 +230,7 @@ ShuffleWriterExec(stage_id=15, output_partitioning=Hash([Column { name: "o_year" CoalesceBatchesExec: target_batch_size=8192 ShuffleReaderExec(stage_id=14, input_partitioning=Hash([Column { name: "o_year", index: 0 }], 2)) -Query Stage #16 (2 -> 1): +Query Stage #16 (1 -> 1): SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] ShuffleReaderExec(stage_id=15, input_partitioning=Hash([Column { name: "o_year", index: 0 }], 2)) diff --git a/testdata/expected-plans/q9.txt b/testdata/expected-plans/q9.txt index 8f738f4..b26aef8 100644 --- a/testdata/expected-plans/q9.txt +++ b/testdata/expected-plans/q9.txt @@ -82,7 +82,7 @@ SortPreservingMergeExec: [nation@0 ASC NULLS LAST,o_year@1 DESC] DataFusion Ray Distributed Plan =========== -Query Stage #0 (1 -> 2): +Query Stage #0 (2 -> 2): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name] @@ -94,11 +94,11 @@ Query Stage #2 (2 -> 2): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }, Column { name: "ps_partkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[ps_partkey, ps_suppkey, ps_supplycost] -Query Stage #3 (1 -> 2): +Query Stage #3 (2 -> 2): ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_nationkey] -Query Stage #4 (1 -> 2): +Query Stage #4 (2 -> 2): ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: p_name@1 LIKE %moccasin%, projection=[p_partkey@0] @@ -166,7 +166,7 @@ ShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "nation" CoalesceBatchesExec: target_batch_size=8192 ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "nation", index: 0 }, Column { name: "o_year", index: 1 }], 2)) -Query Stage #12 (2 -> 1): +Query Stage #12 (1 -> 1): SortPreservingMergeExec: [nation@0 ASC NULLS LAST,o_year@1 DESC] ShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "nation", index: 0 }, Column { name: "o_year", index: 1 }], 2)) From 4e605631ac0ee4a806992e61279cd24754649e4c Mon Sep 17 00:00:00 2001 From: Edmondo Porcu Date: Fri, 13 Dec 2024 18:13:47 -0500 Subject: [PATCH 6/6] Restored test plans for ignored tests --- testdata/expected-plans/q12.txt | 71 +++++++++++++ testdata/expected-plans/q16.txt | 113 ++++++++++++++++++++ testdata/expected-plans/q19.txt | 65 ++++++++++++ testdata/expected-plans/q7.txt | 182 ++++++++++++++++++++++++++++++++ 4 files changed, 431 insertions(+) create mode 100644 testdata/expected-plans/q12.txt create mode 100644 testdata/expected-plans/q16.txt create mode 100644 testdata/expected-plans/q19.txt create mode 100644 testdata/expected-plans/q7.txt diff --git a/testdata/expected-plans/q12.txt b/testdata/expected-plans/q12.txt new file mode 100644 index 0000000..681c4c1 --- /dev/null +++ b/testdata/expected-plans/q12.txt @@ -0,0 +1,71 @@ +DataFusion Logical Plan +======================= + +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)]] + Projection: orders.o_orderpriority, lineitem.l_shipmode + Inner Join: orders.o_orderkey = lineitem.l_orderkey + TableScan: orders projection=[o_orderkey, o_orderpriority] + Projection: lineitem.l_orderkey, lineitem.l_shipmode + Filter: (lineitem.l_shipmode = Utf8("FOB") OR lineitem.l_shipmode = Utf8("SHIP")) AND lineitem.l_receiptdate > lineitem.l_commitdate AND lineitem.l_shipdate < lineitem.l_commitdate AND lineitem.l_receiptdate >= Date32("1995-01-01") AND lineitem.l_receiptdate < Date32("1996-01-01") + TableScan: lineitem projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], partial_filters=[lineitem.l_shipmode = Utf8("FOB") OR lineitem.l_shipmode = Utf8("SHIP"), lineitem.l_receiptdate > lineitem.l_commitdate, lineitem.l_shipdate < lineitem.l_commitdate, lineitem.l_receiptdate >= Date32("1995-01-01"), lineitem.l_receiptdate < Date32("1996-01-01")] + +DataFusion Physical Plan +======================== + +SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] + SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true] + ProjectionExec: expr=[l_shipmode@0 as 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)@1 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)@2 as low_line_count] + AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as 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)] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([l_shipmode@0], 2), input_partitions=2 + AggregateExec: mode=Partial, gby=[l_shipmode@1 as 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)] + ProjectionExec: expr=[o_orderpriority@1 as o_orderpriority, l_shipmode@0 as l_shipmode] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([l_orderkey@0], 2), input_partitions=2 + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: (l_shipmode@4 = FOB OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1995-01-01 AND l_receiptdate@3 < 1996-01-01, projection=[l_orderkey@0, l_shipmode@4] + ParquetExec: file_groups={ ... }, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], predicate=(l_shipmode@14 = FOB OR l_shipmode@14 = SHIP) AND l_receiptdate@12 > l_commitdate@11 AND l_shipdate@10 < l_commitdate@11 AND l_receiptdate@12 >= 1995-01-01 AND l_receiptdate@12 < 1996-01-01, pruning_predicate=(CASE WHEN l_shipmode_null_count@2 = l_shipmode_row_count@3 THEN false ELSE l_shipmode_min@0 <= FOB AND FOB <= l_shipmode_max@1 END OR CASE WHEN l_shipmode_null_count@2 = l_shipmode_row_count@3 THEN false ELSE l_shipmode_min@0 <= SHIP AND SHIP <= l_shipmode_max@1 END) AND CASE WHEN l_receiptdate_null_count@5 = l_receiptdate_row_count@6 THEN false ELSE l_receiptdate_max@4 >= 1995-01-01 END AND CASE WHEN l_receiptdate_null_count@5 = l_receiptdate_row_count@6 THEN false ELSE l_receiptdate_min@7 < 1996-01-01 END, required_guarantees=[l_shipmode in (SHIP, FOB)] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([o_orderkey@0], 2), input_partitions=2 + ParquetExec: file_groups={ ... }, projection=[o_orderkey, o_orderpriority] + +DataFusion Ray Distributed Plan +=========== + +Query Stage #0 (2 -> 2): +ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: (l_shipmode@4 = FOB OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1995-01-01 AND l_receiptdate@3 < 1996-01-01, projection=[l_orderkey@0, l_shipmode@4] + ParquetExec: file_groups={ ... }, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], predicate=(l_shipmode@14 = FOB OR l_shipmode@14 = SHIP) AND l_receiptdate@12 > l_commitdate@11 AND l_shipdate@10 < l_commitdate@11 AND l_receiptdate@12 >= 1995-01-01 AND l_receiptdate@12 < 1996-01-01, pruning_predicate=(CASE WHEN l_shipmode_null_count@2 = l_shipmode_row_count@3 THEN false ELSE l_shipmode_min@0 <= FOB AND FOB <= l_shipmode_max@1 END OR CASE WHEN l_shipmode_null_count@2 = l_shipmode_row_count@3 THEN false ELSE l_shipmode_min@0 <= SHIP AND SHIP <= l_shipmode_max@1 END) AND CASE WHEN l_receiptdate_null_count@5 = l_receiptdate_row_count@6 THEN false ELSE l_receiptdate_max@4 >= 1995-01-01 END AND CASE WHEN l_receiptdate_null_count@5 = l_receiptdate_row_count@6 THEN false ELSE l_receiptdate_min@7 < 1996-01-01 END, required_guarantees=[l_shipmode in (SHIP, FOB)] + +Query Stage #1 (2 -> 2): +ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) + ParquetExec: file_groups={ ... }, projection=[o_orderkey, o_orderpriority] + +Query Stage #2 (2 -> 2): +ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 2)) + AggregateExec: mode=Partial, gby=[l_shipmode@1 as 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)] + ProjectionExec: expr=[o_orderpriority@1 as o_orderpriority, l_shipmode@0 as l_shipmode] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 2)) + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) + +Query Stage #3 (2 -> 2): +ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 2)) + SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true] + ProjectionExec: expr=[l_shipmode@0 as 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)@1 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)@2 as low_line_count] + AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as 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)] + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 2)) + +Query Stage #4 (1 -> 1): +SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] + ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 2)) + diff --git a/testdata/expected-plans/q16.txt b/testdata/expected-plans/q16.txt new file mode 100644 index 0000000..07fb019 --- /dev/null +++ b/testdata/expected-plans/q16.txt @@ -0,0 +1,113 @@ +DataFusion Logical Plan +======================= + +Sort: supplier_cnt DESC NULLS FIRST, part.p_brand ASC NULLS LAST, part.p_type ASC NULLS LAST, part.p_size ASC NULLS LAST + Projection: part.p_brand, part.p_type, part.p_size, count(alias1) AS supplier_cnt + Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size]], aggr=[[count(alias1)]] + Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size, partsupp.ps_suppkey AS alias1]], aggr=[[]] + LeftAnti Join: partsupp.ps_suppkey = __correlated_sq_1.s_suppkey + 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#14") AND part.p_type NOT LIKE Utf8("SMALL PLATED%") AND part.p_size IN ([Int32(14), Int32(6), Int32(5), Int32(31), Int32(49), Int32(15), Int32(41), Int32(47)]) + TableScan: part projection=[p_partkey, p_brand, p_type, p_size], partial_filters=[part.p_brand != Utf8("Brand#14"), part.p_type NOT LIKE Utf8("SMALL PLATED%"), part.p_size IN ([Int32(14), Int32(6), Int32(5), Int32(31), Int32(49), Int32(15), Int32(41), Int32(47)])] + SubqueryAlias: __correlated_sq_1 + Projection: supplier.s_suppkey + Filter: supplier.s_comment LIKE Utf8("%Customer%Complaints%") + TableScan: supplier projection=[s_suppkey, s_comment], partial_filters=[supplier.s_comment LIKE Utf8("%Customer%Complaints%")] + +DataFusion Physical Plan +======================== + +SortPreservingMergeExec: [supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST] + SortExec: expr=[supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST], preserve_partitioning=[true] + ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, count(alias1)@3 as supplier_cnt] + AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 2), input_partitions=2 + AggregateExec: mode=Partial, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] + AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, alias1@3 as alias1], aggr=[] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 2), input_partitions=2 + AggregateExec: mode=Partial, gby=[p_brand@1 as p_brand, p_type@2 as p_type, p_size@3 as p_size, ps_suppkey@0 as alias1], aggr=[] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=RightAnti, on=[(s_suppkey@0, ps_suppkey@0)] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([s_suppkey@0], 2), input_partitions=2 + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] + RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 + ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_comment], predicate=s_comment@6 LIKE %Customer%Complaints% + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([ps_suppkey@0], 2), input_partitions=2 + ProjectionExec: expr=[ps_suppkey@3 as ps_suppkey, p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_brand@1, p_type@2, p_size@3, ps_suppkey@5] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([p_partkey@0], 2), input_partitions=2 + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: p_brand@1 != Brand#14 AND p_type@2 NOT LIKE SMALL PLATED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(14) }, Literal { value: Int32(6) }, Literal { value: Int32(5) }, Literal { value: Int32(31) }, Literal { value: Int32(49) }, Literal { value: Int32(15) }, Literal { value: Int32(41) }, Literal { value: Int32(47) }]) + RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 + ParquetExec: file_groups={ ... }]), pruning_predicate=CASE WHEN p_brand_null_count@2 = p_brand_row_count@3 THEN false ELSE p_brand_min@0 != Brand#14 OR Brand#14 != p_brand_max@1 END AND (CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 14 AND 14 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 6 AND 6 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 5 AND 5 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 31 AND 31 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 49 AND 49 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 15 AND 15 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 41 AND 41 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 47 AND 47 <= p_size_max@5 END), required_guarantees=[p_brand not in (Brand#14), p_size in (6, 5, 31, 41, 47, 14, 15, 49)] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([ps_partkey@0], 2), input_partitions=2 + ParquetExec: file_groups={ ... }, projection=[ps_partkey, ps_suppkey] + +DataFusion Ray Distributed Plan +=========== + +Query Stage #0 (2 -> 2): +ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] + ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_comment], predicate=s_comment@6 LIKE %Customer%Complaints% + +Query Stage #1 (2 -> 2): +ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: p_brand@1 != Brand#14 AND p_type@2 NOT LIKE SMALL PLATED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(14) }, Literal { value: Int32(6) }, Literal { value: Int32(5) }, Literal { value: Int32(31) }, Literal { value: Int32(49) }, Literal { value: Int32(15) }, Literal { value: Int32(41) }, Literal { value: Int32(47) }]) + ParquetExec: file_groups={ ... }]), pruning_predicate=CASE WHEN p_brand_null_count@2 = p_brand_row_count@3 THEN false ELSE p_brand_min@0 != Brand#14 OR Brand#14 != p_brand_max@1 END AND (CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 14 AND 14 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 6 AND 6 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 5 AND 5 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 31 AND 31 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 49 AND 49 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 15 AND 15 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 41 AND 41 <= p_size_max@5 END OR CASE WHEN p_size_null_count@6 = p_size_row_count@7 THEN false ELSE p_size_min@4 <= 47 AND 47 <= p_size_max@5 END), required_guarantees=[p_brand not in (Brand#14), p_size in (6, 5, 31, 41, 47, 14, 15, 49)] + +Query Stage #2 (2 -> 2): +ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) + ParquetExec: file_groups={ ... }, projection=[ps_partkey, ps_suppkey] + +Query Stage #3 (2 -> 2): +ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 2)) + ProjectionExec: expr=[ps_suppkey@3 as ps_suppkey, p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_brand@1, p_type@2, p_size@3, ps_suppkey@5] + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 2)) + +Query Stage #4 (2 -> 2): +ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "p_brand", index: 0 }, Column { name: "p_type", index: 1 }, Column { name: "p_size", index: 2 }, Column { name: "alias1", index: 3 }], 2)) + AggregateExec: mode=Partial, gby=[p_brand@1 as p_brand, p_type@2 as p_type, p_size@3 as p_size, ps_suppkey@0 as alias1], aggr=[] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=RightAnti, on=[(s_suppkey@0, ps_suppkey@0)] + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 2)) + +Query Stage #5 (2 -> 2): +ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "p_brand", index: 0 }, Column { name: "p_type", index: 1 }, Column { name: "p_size", index: 2 }], 2)) + AggregateExec: mode=Partial, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] + AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, alias1@3 as alias1], aggr=[] + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "p_brand", index: 0 }, Column { name: "p_type", index: 1 }, Column { name: "p_size", index: 2 }, Column { name: "alias1", index: 3 }], 2)) + +Query Stage #6 (2 -> 2): +ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "p_brand", index: 0 }, Column { name: "p_type", index: 1 }, Column { name: "p_size", index: 2 }], 2)) + SortExec: expr=[supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST], preserve_partitioning=[true] + ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, count(alias1)@3 as supplier_cnt] + AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "p_brand", index: 0 }, Column { name: "p_type", index: 1 }, Column { name: "p_size", index: 2 }], 2)) + +Query Stage #7 (1 -> 1): +SortPreservingMergeExec: [supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST] + ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "p_brand", index: 0 }, Column { name: "p_type", index: 1 }, Column { name: "p_size", index: 2 }], 2)) + diff --git a/testdata/expected-plans/q19.txt b/testdata/expected-plans/q19.txt new file mode 100644 index 0000000..7b1067d --- /dev/null +++ b/testdata/expected-plans/q19.txt @@ -0,0 +1,65 @@ +DataFusion Logical Plan +======================= + +Projection: sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue + Aggregate: groupBy=[[]], aggr=[[sum(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] + Projection: lineitem.l_extendedprice, lineitem.l_discount + Inner Join: lineitem.l_partkey = part.p_partkey Filter: part.p_brand = Utf8("Brand#21") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND lineitem.l_quantity >= Decimal128(Some(800),11,2) AND lineitem.l_quantity <= Decimal128(Some(1800),11,2) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#13") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND lineitem.l_quantity >= Decimal128(Some(2000),11,2) AND lineitem.l_quantity <= Decimal128(Some(3000),11,2) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#52") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND lineitem.l_quantity >= Decimal128(Some(3000),11,2) AND lineitem.l_quantity <= Decimal128(Some(4000),11,2) AND part.p_size <= Int32(15) + Projection: lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount + Filter: (lineitem.l_quantity >= Decimal128(Some(800),11,2) AND lineitem.l_quantity <= Decimal128(Some(1800),11,2) OR lineitem.l_quantity >= Decimal128(Some(2000),11,2) AND lineitem.l_quantity <= Decimal128(Some(3000),11,2) OR lineitem.l_quantity >= Decimal128(Some(3000),11,2) AND lineitem.l_quantity <= Decimal128(Some(4000),11,2)) AND (lineitem.l_shipmode = Utf8("AIR") OR lineitem.l_shipmode = Utf8("AIR REG")) AND lineitem.l_shipinstruct = Utf8("DELIVER IN PERSON") + TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], partial_filters=[lineitem.l_shipmode = Utf8("AIR") OR lineitem.l_shipmode = Utf8("AIR REG"), lineitem.l_shipinstruct = Utf8("DELIVER IN PERSON"), lineitem.l_quantity >= Decimal128(Some(800),11,2) AND lineitem.l_quantity <= Decimal128(Some(1800),11,2) OR lineitem.l_quantity >= Decimal128(Some(2000),11,2) AND lineitem.l_quantity <= Decimal128(Some(3000),11,2) OR lineitem.l_quantity >= Decimal128(Some(3000),11,2) AND lineitem.l_quantity <= Decimal128(Some(4000),11,2)] + Filter: (part.p_brand = Utf8("Brand#21") 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#13") 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#52") 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], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8("Brand#21") 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#13") 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#52") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND part.p_size <= Int32(15)] + +DataFusion Physical Plan +======================== + +ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue] + AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + CoalescePartitionsExec + AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], filter=p_brand@1 = Brand#21 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND l_quantity@0 >= Some(800),11,2 AND l_quantity@0 <= Some(1800),11,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#13 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND l_quantity@0 >= Some(2000),11,2 AND l_quantity@0 <= Some(3000),11,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#52 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND l_quantity@0 >= Some(3000),11,2 AND l_quantity@0 <= Some(4000),11,2 AND p_size@2 <= 15, projection=[l_extendedprice@6, l_discount@7] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([p_partkey@0], 2), input_partitions=2 + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: (p_brand@1 = Brand#21 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#13 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#52 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND p_size@2 <= 15) AND p_size@2 >= 1 + RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 + ParquetExec: file_groups={ ... }]) AND p_size@5 <= 15), pruning_predicate=CASE WHEN p_size_null_count@1 = p_size_row_count@2 THEN false ELSE p_size_max@0 >= 1 END AND (CASE WHEN p_brand_null_count@5 = p_brand_row_count@6 THEN false ELSE p_brand_min@3 <= Brand#21 AND Brand#21 <= p_brand_max@4 END AND (CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= SM CASE AND SM CASE <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= SM BOX AND SM BOX <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= SM PACK AND SM PACK <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= SM PKG AND SM PKG <= p_container_max@8 END) AND CASE WHEN p_size_null_count@1 = p_size_row_count@2 THEN false ELSE p_size_min@11 <= 5 END OR CASE WHEN p_brand_null_count@5 = p_brand_row_count@6 THEN false ELSE p_brand_min@3 <= Brand#13 AND Brand#13 <= p_brand_max@4 END AND (CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= MED BAG AND MED BAG <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= MED BOX AND MED BOX <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= MED PKG AND MED PKG <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= MED PACK AND MED PACK <= p_container_max@8 END) AND CASE WHEN p_size_null_count@1 = p_size_row_count@2 THEN false ELSE p_size_min@11 <= 10 END OR CASE WHEN p_brand_null_count@5 = p_brand_row_count@6 THEN false ELSE p_brand_min@3 <= Brand#52 AND Brand#52 <= p_brand_max@4 END AND (CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= LG CASE AND LG CASE <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= LG BOX AND LG BOX <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= LG PACK AND LG PACK <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= LG PKG AND LG PKG <= p_container_max@8 END) AND CASE WHEN p_size_null_count@1 = p_size_row_count@2 THEN false ELSE p_size_min@11 <= 15 END), required_guarantees=[] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([l_partkey@0], 2), input_partitions=2 + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: (l_quantity@1 >= Some(800),11,2 AND l_quantity@1 <= Some(1800),11,2 OR l_quantity@1 >= Some(2000),11,2 AND l_quantity@1 <= Some(3000),11,2 OR l_quantity@1 >= Some(3000),11,2 AND l_quantity@1 <= Some(4000),11,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON, projection=[l_partkey@0, l_quantity@1, l_extendedprice@2, l_discount@3] + ParquetExec: file_groups={ ... }, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], predicate=(l_shipmode@14 = AIR OR l_shipmode@14 = AIR REG) AND l_shipinstruct@13 = DELIVER IN PERSON AND (l_quantity@4 >= Some(800),11,2 AND l_quantity@4 <= Some(1800),11,2 OR l_quantity@4 >= Some(2000),11,2 AND l_quantity@4 <= Some(3000),11,2 OR l_quantity@4 >= Some(3000),11,2 AND l_quantity@4 <= Some(4000),11,2), pruning_predicate=(CASE WHEN l_shipmode_null_count@2 = l_shipmode_row_count@3 THEN false ELSE l_shipmode_min@0 <= AIR AND AIR <= l_shipmode_max@1 END OR CASE WHEN l_shipmode_null_count@2 = l_shipmode_row_count@3 THEN false ELSE l_shipmode_min@0 <= AIR REG AND AIR REG <= l_shipmode_max@1 END) AND CASE WHEN l_shipinstruct_null_count@6 = l_shipinstruct_row_count@7 THEN false ELSE l_shipinstruct_min@4 <= DELIVER IN PERSON AND DELIVER IN PERSON <= l_shipinstruct_max@5 END AND (CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_max@8 >= Some(800),11,2 END AND CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_min@11 <= Some(1800),11,2 END OR CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_max@8 >= Some(2000),11,2 END AND CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_min@11 <= Some(3000),11,2 END OR CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_max@8 >= Some(3000),11,2 END AND CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_min@11 <= Some(4000),11,2 END), required_guarantees=[l_shipmode in (AIR REG, AIR), l_shipinstruct in (DELIVER IN PERSON)] + +DataFusion Ray Distributed Plan +=========== + +Query Stage #0 (2 -> 2): +ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: (p_brand@1 = Brand#21 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#13 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#52 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND p_size@2 <= 15) AND p_size@2 >= 1 + ParquetExec: file_groups={ ... }]) AND p_size@5 <= 15), pruning_predicate=CASE WHEN p_size_null_count@1 = p_size_row_count@2 THEN false ELSE p_size_max@0 >= 1 END AND (CASE WHEN p_brand_null_count@5 = p_brand_row_count@6 THEN false ELSE p_brand_min@3 <= Brand#21 AND Brand#21 <= p_brand_max@4 END AND (CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= SM CASE AND SM CASE <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= SM BOX AND SM BOX <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= SM PACK AND SM PACK <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= SM PKG AND SM PKG <= p_container_max@8 END) AND CASE WHEN p_size_null_count@1 = p_size_row_count@2 THEN false ELSE p_size_min@11 <= 5 END OR CASE WHEN p_brand_null_count@5 = p_brand_row_count@6 THEN false ELSE p_brand_min@3 <= Brand#13 AND Brand#13 <= p_brand_max@4 END AND (CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= MED BAG AND MED BAG <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= MED BOX AND MED BOX <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= MED PKG AND MED PKG <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= MED PACK AND MED PACK <= p_container_max@8 END) AND CASE WHEN p_size_null_count@1 = p_size_row_count@2 THEN false ELSE p_size_min@11 <= 10 END OR CASE WHEN p_brand_null_count@5 = p_brand_row_count@6 THEN false ELSE p_brand_min@3 <= Brand#52 AND Brand#52 <= p_brand_max@4 END AND (CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= LG CASE AND LG CASE <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= LG BOX AND LG BOX <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= LG PACK AND LG PACK <= p_container_max@8 END OR CASE WHEN p_container_null_count@9 = p_container_row_count@10 THEN false ELSE p_container_min@7 <= LG PKG AND LG PKG <= p_container_max@8 END) AND CASE WHEN p_size_null_count@1 = p_size_row_count@2 THEN false ELSE p_size_min@11 <= 15 END), required_guarantees=[] + +Query Stage #1 (2 -> 2): +ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 2)) + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: (l_quantity@1 >= Some(800),11,2 AND l_quantity@1 <= Some(1800),11,2 OR l_quantity@1 >= Some(2000),11,2 AND l_quantity@1 <= Some(3000),11,2 OR l_quantity@1 >= Some(3000),11,2 AND l_quantity@1 <= Some(4000),11,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON, projection=[l_partkey@0, l_quantity@1, l_extendedprice@2, l_discount@3] + ParquetExec: file_groups={ ... }, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], predicate=(l_shipmode@14 = AIR OR l_shipmode@14 = AIR REG) AND l_shipinstruct@13 = DELIVER IN PERSON AND (l_quantity@4 >= Some(800),11,2 AND l_quantity@4 <= Some(1800),11,2 OR l_quantity@4 >= Some(2000),11,2 AND l_quantity@4 <= Some(3000),11,2 OR l_quantity@4 >= Some(3000),11,2 AND l_quantity@4 <= Some(4000),11,2), pruning_predicate=(CASE WHEN l_shipmode_null_count@2 = l_shipmode_row_count@3 THEN false ELSE l_shipmode_min@0 <= AIR AND AIR <= l_shipmode_max@1 END OR CASE WHEN l_shipmode_null_count@2 = l_shipmode_row_count@3 THEN false ELSE l_shipmode_min@0 <= AIR REG AND AIR REG <= l_shipmode_max@1 END) AND CASE WHEN l_shipinstruct_null_count@6 = l_shipinstruct_row_count@7 THEN false ELSE l_shipinstruct_min@4 <= DELIVER IN PERSON AND DELIVER IN PERSON <= l_shipinstruct_max@5 END AND (CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_max@8 >= Some(800),11,2 END AND CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_min@11 <= Some(1800),11,2 END OR CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_max@8 >= Some(2000),11,2 END AND CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_min@11 <= Some(3000),11,2 END OR CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_max@8 >= Some(3000),11,2 END AND CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_min@11 <= Some(4000),11,2 END), required_guarantees=[l_shipmode in (AIR REG, AIR), l_shipinstruct in (DELIVER IN PERSON)] + +Query Stage #2 (2 -> 1): +ShuffleWriterExec(stage_id=2, output_partitioning=Hash([], 2)) + AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], filter=p_brand@1 = Brand#21 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND l_quantity@0 >= Some(800),11,2 AND l_quantity@0 <= Some(1800),11,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#13 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND l_quantity@0 >= Some(2000),11,2 AND l_quantity@0 <= Some(3000),11,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#52 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND l_quantity@0 >= Some(3000),11,2 AND l_quantity@0 <= Some(4000),11,2 AND p_size@2 <= 15, projection=[l_extendedprice@6, l_discount@7] + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 2)) + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 2)) + +Query Stage #3 (1 -> 1): +ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue] + AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + CoalescePartitionsExec + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([], 2)) + diff --git a/testdata/expected-plans/q7.txt b/testdata/expected-plans/q7.txt new file mode 100644 index 0000000..37e3b27 --- /dev/null +++ b/testdata/expected-plans/q7.txt @@ -0,0 +1,182 @@ +DataFusion Logical Plan +======================= + +Sort: shipping.supp_nation ASC NULLS LAST, shipping.cust_nation ASC NULLS LAST, shipping.l_year ASC NULLS LAST + Projection: shipping.supp_nation, shipping.cust_nation, shipping.l_year, sum(shipping.volume) AS revenue + Aggregate: groupBy=[[shipping.supp_nation, shipping.cust_nation, shipping.l_year]], aggr=[[sum(shipping.volume)]] + SubqueryAlias: shipping + Projection: n1.n_name AS supp_nation, n2.n_name AS cust_nation, date_part(Utf8("YEAR"), lineitem.l_shipdate) AS l_year, lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS volume + Inner Join: customer.c_nationkey = n2.n_nationkey Filter: n1.n_name = Utf8("GERMANY") AND n2.n_name = Utf8("IRAQ") OR n1.n_name = Utf8("IRAQ") AND n2.n_name = Utf8("GERMANY") + 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("1995-01-01") AND lineitem.l_shipdate <= Date32("1996-12-31") + TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("1995-01-01"), lineitem.l_shipdate <= Date32("1996-12-31")] + TableScan: orders projection=[o_orderkey, o_custkey] + TableScan: customer projection=[c_custkey, c_nationkey] + SubqueryAlias: n1 + Filter: nation.n_name = Utf8("GERMANY") OR nation.n_name = Utf8("IRAQ") + TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("GERMANY") OR nation.n_name = Utf8("IRAQ")] + SubqueryAlias: n2 + Filter: nation.n_name = Utf8("IRAQ") OR nation.n_name = Utf8("GERMANY") + TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("IRAQ") OR nation.n_name = Utf8("GERMANY")] + +DataFusion Physical Plan +======================== + +SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST] + SortExec: expr=[supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST], preserve_partitioning=[true] + ProjectionExec: expr=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year, sum(shipping.volume)@3 as revenue] + AggregateExec: mode=FinalPartitioned, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 2), input_partitions=2 + AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] + ProjectionExec: expr=[n_name@4 as supp_nation, n_name@0 as cust_nation, date_part(YEAR, l_shipdate@3) as l_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as volume] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_nationkey@0, c_nationkey@3)], filter=n_name@0 = GERMANY AND n_name@1 = IRAQ OR n_name@0 = IRAQ AND n_name@1 = GERMANY, projection=[n_name@1, l_extendedprice@2, l_discount@3, l_shipdate@4, n_name@6] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([n_nationkey@0], 2), input_partitions=2 + RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: n_name@1 = IRAQ OR n_name@1 = GERMANY + ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name], predicate=n_name@1 = IRAQ OR n_name@1 = GERMANY, pruning_predicate=CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= IRAQ AND IRAQ <= n_name_max@1 END OR CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1 END, required_guarantees=[n_name in (GERMANY, IRAQ)] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([c_nationkey@3], 2), input_partitions=2 + ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@4 as c_nationkey, n_name@0 as n_name] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_nationkey@0, s_nationkey@0)], projection=[n_name@1, l_extendedprice@3, l_discount@4, l_shipdate@5, c_nationkey@6] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([n_nationkey@0], 2), input_partitions=2 + RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: n_name@1 = GERMANY OR n_name@1 = IRAQ + ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name], predicate=n_name@1 = GERMANY OR n_name@1 = IRAQ, pruning_predicate=CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1 END OR CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= IRAQ AND IRAQ <= n_name_max@1 END, required_guarantees=[n_name in (GERMANY, IRAQ)] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([s_nationkey@0], 2), input_partitions=2 + ProjectionExec: expr=[s_nationkey@1 as s_nationkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, l_shipdate@4 as l_shipdate, c_nationkey@0 as c_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@4)], projection=[c_nationkey@1, s_nationkey@2, l_extendedprice@3, l_discount@4, l_shipdate@5] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([c_custkey@0], 2), input_partitions=2 + ParquetExec: file_groups={ ... }, projection=[c_custkey, c_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([o_custkey@4], 2), input_partitions=2 + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([l_orderkey@1], 2), input_partitions=2 + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([s_suppkey@0], 2), input_partitions=2 + RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 + ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([l_suppkey@1], 2), input_partitions=2 + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 + ParquetExec: file_groups={ ... }, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], predicate=l_shipdate@10 >= 1995-01-01 AND l_shipdate@10 <= 1996-12-31, pruning_predicate=CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_max@0 >= 1995-01-01 END AND CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_min@3 <= 1996-12-31 END, required_guarantees=[] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([o_orderkey@0], 2), input_partitions=2 + ParquetExec: file_groups={ ... }, projection=[o_orderkey, o_custkey] + +DataFusion Ray Distributed Plan +=========== + +Query Stage #0 (2 -> 2): +ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: n_name@1 = IRAQ OR n_name@1 = GERMANY + ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name], predicate=n_name@1 = IRAQ OR n_name@1 = GERMANY, pruning_predicate=CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= IRAQ AND IRAQ <= n_name_max@1 END OR CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1 END, required_guarantees=[n_name in (GERMANY, IRAQ)] + +Query Stage #1 (2 -> 2): +ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: n_name@1 = GERMANY OR n_name@1 = IRAQ + ParquetExec: file_groups={ ... }, projection=[n_nationkey, n_name], predicate=n_name@1 = GERMANY OR n_name@1 = IRAQ, pruning_predicate=CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1 END OR CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= IRAQ AND IRAQ <= n_name_max@1 END, required_guarantees=[n_name in (GERMANY, IRAQ)] + +Query Stage #2 (2 -> 2): +ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) + ParquetExec: file_groups={ ... }, projection=[c_custkey, c_nationkey] + +Query Stage #3 (2 -> 2): +ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) + ParquetExec: file_groups={ ... }, projection=[s_suppkey, s_nationkey] + +Query Stage #4 (2 -> 2): +ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 2)) + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 + ParquetExec: file_groups={ ... }, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], predicate=l_shipdate@10 >= 1995-01-01 AND l_shipdate@10 <= 1996-12-31, pruning_predicate=CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_max@0 >= 1995-01-01 END AND CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_min@3 <= 1996-12-31 END, required_guarantees=[] + +Query Stage #5 (2 -> 2): +ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "l_orderkey", index: 1 }], 2)) + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 2)) + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 2)) + +Query Stage #6 (2 -> 2): +ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) + ParquetExec: file_groups={ ... }, projection=[o_orderkey, o_custkey] + +Query Stage #7 (2 -> 2): +ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "o_custkey", index: 4 }], 2)) + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6] + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "l_orderkey", index: 1 }], 2)) + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 2)) + +Query Stage #8 (2 -> 2): +ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "s_nationkey", index: 0 }], 2)) + ProjectionExec: expr=[s_nationkey@1 as s_nationkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, l_shipdate@4 as l_shipdate, c_nationkey@0 as c_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@4)], projection=[c_nationkey@1, s_nationkey@2, l_extendedprice@3, l_discount@4, l_shipdate@5] + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "o_custkey", index: 4 }], 2)) + +Query Stage #9 (2 -> 2): +ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "c_nationkey", index: 3 }], 2)) + ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@4 as c_nationkey, n_name@0 as n_name] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_nationkey@0, s_nationkey@0)], projection=[n_name@1, l_extendedprice@3, l_discount@4, l_shipdate@5, c_nationkey@6] + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "s_nationkey", index: 0 }], 2)) + +Query Stage #10 (2 -> 2): +ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "supp_nation", index: 0 }, Column { name: "cust_nation", index: 1 }, Column { name: "l_year", index: 2 }], 2)) + AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] + ProjectionExec: expr=[n_name@4 as supp_nation, n_name@0 as cust_nation, date_part(YEAR, l_shipdate@3) as l_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as volume] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_nationkey@0, c_nationkey@3)], filter=n_name@0 = GERMANY AND n_name@1 = IRAQ OR n_name@0 = IRAQ AND n_name@1 = GERMANY, projection=[n_name@1, l_extendedprice@2, l_discount@3, l_shipdate@4, n_name@6] + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 2)) + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "c_nationkey", index: 3 }], 2)) + +Query Stage #11 (2 -> 2): +ShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "supp_nation", index: 0 }, Column { name: "cust_nation", index: 1 }, Column { name: "l_year", index: 2 }], 2)) + SortExec: expr=[supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST], preserve_partitioning=[true] + ProjectionExec: expr=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year, sum(shipping.volume)@3 as revenue] + AggregateExec: mode=FinalPartitioned, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "supp_nation", index: 0 }, Column { name: "cust_nation", index: 1 }, Column { name: "l_year", index: 2 }], 2)) + +Query Stage #12 (1 -> 1): +SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST] + ShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "supp_nation", index: 0 }, Column { name: "cust_nation", index: 1 }, Column { name: "l_year", index: 2 }], 2)) +