diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 66d4aeb..7ac8057 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -2,24 +2,36 @@ name: Rust on: push: - branches: [ "main" ] pull_request: - branches: [ "main" ] env: CARGO_TERM_COLOR: always + PYTHON_VERSION: 3.9 + TPCH_SCALING_FACTOR: "1" + TPCH_TEST_PARTITIONS: "1" + TPCH_DATA_PATH: "data" jobs: build: - runs-on: ubuntu-latest steps: - - uses: actions/checkout@v3 - - name: Install protobuf compiler - shell: bash - run: sudo apt-get install protobuf-compiler - - name: Build Rust code - run: cargo build --verbose - - name: Run tests - run: cargo test --verbose + - uses: actions/checkout@v3 + - 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 + with: + python-version: ${{ env.PYTHON_VERSION }} + - name: Install test dependencies + run: | + python -m pip install --upgrade pip + pip install -r tpch/requirements.txt + - name: Generate test data + run: | + ./scripts/gen-test-data.sh + - name: Run tests + run: cargo test --verbose diff --git a/.gitignore b/.gitignore index 0c56375..fdbe035 100644 --- a/.gitignore +++ b/.gitignore @@ -5,3 +5,5 @@ venv *.so *.log results-sf* +data +tpch/tpch-dbgen diff --git a/Cargo.lock b/Cargo.lock index aa6cdff..c0bea07 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1115,21 +1115,30 @@ dependencies = [ name = "datafusion_ray" version = "0.6.0" dependencies = [ + "anyhow", "datafusion", "datafusion-proto", "datafusion-python", "futures", "glob", "log", + "pretty_assertions", "prost 0.12.6", "prost-types 0.12.6", "pyo3", + "regex", "rustc_version", "tokio", "tonic-build", "uuid", ] +[[package]] +name = "diff" +version = "0.1.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "56254986775e3233ffa9c4d7d3faaf6d36a2c09d30b20687e9f88bc8bafc16c8" + [[package]] name = "digest" version = "0.10.7" @@ -2190,6 +2199,16 @@ dependencies = [ "zerocopy", ] +[[package]] +name = "pretty_assertions" +version = "1.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3ae130e2f271fbc2ac3a40fb1d07180839cdbbe443c7a27e1e3c13c5cac0116d" +dependencies = [ + "diff", + "yansi", +] + [[package]] name = "prettyplease" version = "0.1.25" @@ -2472,9 +2491,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.10.6" +version = "1.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4219d74c6b67a3654a9fbebc4b419e22126d13d2f3c4a07ee0cb61ff79a79619" +checksum = "38200e5ee88914975b69f657f0801b6f6dccafd44fd9326302a4aaeecfacb1d8" dependencies = [ "aho-corasick", "memchr", @@ -2484,9 +2503,9 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.7" +version = "0.4.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38caf58cc5ef2fed281f89292ef23f6365465ed9a41b7a7754eb4e26496c92df" +checksum = "368758f23274712b504848e9d5a6f010445cc8b87a7cdb4d7cbee666c1288da3" dependencies = [ "aho-corasick", "memchr", @@ -2501,9 +2520,9 @@ checksum = "53a49587ad06b26609c52e423de037e7f57f20d53535d66e08c695f347df952a" [[package]] name = "regex-syntax" -version = "0.8.4" +version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a66a03ae7c801facd77a29370b4faec201768915ac14a721ba36f20bc9c209b" +checksum = "2b15c43186be67a4fd63bee50d0303afffcef381492ebe2c5d87f324e1b8815c" [[package]] name = "reqwest" @@ -3550,6 +3569,12 @@ dependencies = [ "lzma-sys", ] +[[package]] +name = "yansi" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cfe53a6657fd280eaa890a3bc59152892ffa3e30101319d168b781ed6529b049" + [[package]] name = "zerocopy" version = "0.7.35" diff --git a/Cargo.toml b/Cargo.toml index 081520f..53bb6c0 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -45,6 +45,11 @@ uuid = "1.2" rustc_version = "0.4.0" tonic-build = { version = "0.8", default-features = false, features = ["transport", "prost"] } +[dev-dependencies] +anyhow = "1.0.89" +pretty_assertions = "1.4.0" +regex = "1.11.0" + [lib] name = "datafusion_ray" crate-type = ["cdylib", "rlib"] @@ -54,4 +59,4 @@ name = "datafusion_ray._datafusion_ray_internal" [profile.release] codegen-units = 1 -lto = true \ No newline at end of file +lto = true diff --git a/README.md b/README.md index 5aa86e0..b3576fc 100644 --- a/README.md +++ b/README.md @@ -19,8 +19,8 @@ # DataFusion on Ray -> This was originally a research project donated from [ray-sql](https://github.com/datafusion-contrib/ray-sql) to evaluate performing distributed SQL queries from Python, using -[Ray](https://www.ray.io/) and [DataFusion](https://github.com/apache/arrow-datafusion). +> This was originally a research project donated from [ray-sql](https://github.com/datafusion-contrib/ray-sql) to evaluate performing distributed SQL queries from Python, using +> [Ray](https://www.ray.io/) and [DataFusion](https://github.com/apache/arrow-datafusion). DataFusion Ray is a distributed SQL query engine powered by the Rust implementation of [Apache Arrow](https://arrow.apache.org/), [Apache DataFusion](https://datafusion.apache.org/) and [Ray](https://www.ray.io/). @@ -33,7 +33,7 @@ DataFusion Ray is a distributed SQL query engine powered by the Rust implementat ## Non Goals -- Re-build the cluster scheduling systems like what [Ballista](https://datafusion.apache.org/ballista/) did. +- Re-build the cluster scheduling systems like what [Ballista](https://datafusion.apache.org/ballista/) did. - Ballista is extremely complex and utilizing Ray feels like it abstracts some of that complexity away. - Datafusion Ray is delegating cluster management to Ray. @@ -120,10 +120,38 @@ python -m pip install -r requirements-in.txt Whenever rust code changes (your changes or via `git pull`): -```bash +````bash # make sure you activate the venv using "source venv/bin/activate" first -maturin develop -python -m pytest +maturin develop python -m pytest ``` + + +## Testing + +Running local Rust tests require generating the tpch-data. This can be done +by running the following command: + +```bash +./scripts/generate_tpch_data.sh +``` + +Tests compare plans with expected plans, which unfortunately contain the +path to the parquet tables. The path committed under version control is +the one of a Github Runner, and won't work locally. You can fix it by +running the following command: + +```bash +./scripts/replace-expected-plan-paths.sh local-dev +```` + +When instead you need to regenerate the plans, which you can do by +re-running the planner tests removing all the content of +`testdata/expected-plans`, they will now contain your local paths. You can +fix it before committing the plans running + +```bash + +./scripts/replace-expected-plan-paths.sh pre-ci + ``` ## Benchmarking diff --git a/scripts/gen-test-data.sh b/scripts/gen-test-data.sh new file mode 100755 index 0000000..a46a01f --- /dev/null +++ b/scripts/gen-test-data.sh @@ -0,0 +1,60 @@ +#!/bin/bash + +set -e + +create_directories() { + mkdir -p data +} + +clone_and_build_tpch_dbgen() { + if [ -z "$(ls -A tpch/tpch-dbgen)" ]; then + echo "tpch/tpch-dbgen folder is empty. Cloning repository..." + git clone https://github.com/databricks/tpch-dbgen.git tpch/tpch-dbgen + cd tpch/tpch-dbgen + make + cd ../../ + else + echo "tpch/tpch-dbgen folder is not empty. Skipping cloning of TPCH dbgen." + fi +} + +generate_data() { + cd tpch/tpch-dbgen + if [ "$TPCH_TEST_PARTITIONS" -gt 1 ]; then + for i in $(seq 1 "$TPCH_TEST_PARTITIONS"); do + ./dbgen -f -s "$TPCH_SCALING_FACTOR" -C "$TPCH_TEST_PARTITIONS" -S "$i" + done + else + ./dbgen -f -s "$TPCH_SCALING_FACTOR" + fi + mv ./*.tbl* ../../data +} + +convert_data() { + cd ../../ + python -m tpch.tpchgen convert --partitions "$TPCH_TEST_PARTITIONS" +} + +main() { + if [ -z "$TPCH_TEST_PARTITIONS" ]; then + echo "Error: TPCH_TEST_PARTITIONS is not set." + exit 1 + fi + + if [ -z "$TPCH_SCALING_FACTOR" ]; then + echo "Error: TPCH_SCALING_FACTOR is not set." + exit 1 + fi + + create_directories + + if [ -z "$(ls -A data)" ]; then + clone_and_build_tpch_dbgen + generate_data + convert_data + else + echo "Data folder is not empty. Skipping cloning and data generation." + fi +} + +main diff --git a/scripts/replace-expected-plans-paths.sh b/scripts/replace-expected-plans-paths.sh new file mode 100755 index 0000000..527fd97 --- /dev/null +++ b/scripts/replace-expected-plans-paths.sh @@ -0,0 +1,44 @@ +#!/bin/bash + +# This script helps change the path to parquet files in expected plans for +# local development and CI + +set -e + +if [ "$#" -ne 1 ]; then + echo "Usage: $0 " + echo "Modes: pre-ci, local-dev" + exit 1 +fi + +# Assign the parameter to the mode variable +mode=$1 + +ci_dir="home/runner/work/datafusion-ray/datafusion-ray" +current_dir=$(pwd) +current_dir_no_leading_slash="${current_dir#/}" +expected_plans_dir="./testdata/expected-plans" + +# Function to replace paths in files +replace_paths() { + local search=$1 + local replace=$2 + find "$expected_plans_dir" -type f -exec sed -i "s|$search|$replace|g" {} + + echo "Replaced all occurrences of '$search' with '$replace' in files within '$expected_plans_dir'." +} + +# Handle the modes +case $mode in +pre-ci) + replace_paths "$current_dir_no_leading_slash" "$ci_dir" + ;; +local-dev) + replace_paths "$ci_dir" "$current_dir_no_leading_slash" + ;; +*) + echo "Invalid mode: $mode" + echo "Usage: $0 " + echo "Modes: pre-ci, local-dev" + exit 1 + ;; +esac diff --git a/src/planner.rs b/src/planner.rs index b940c9d..0d30552 100644 --- a/src/planner.rs +++ b/src/planner.rs @@ -264,128 +264,135 @@ mod test { use super::*; use datafusion::physical_plan::displayable; use datafusion::prelude::{ParquetReadOptions, SessionConfig, SessionContext}; - use std::fs; + use pretty_assertions::assert_eq; + use regex::Regex; use std::path::Path; + use std::{env, fs}; + type TestResult = std::result::Result; #[tokio::test] - async fn test_q1() -> Result<()> { + async fn test_q1() -> TestResult<()> { do_test(1).await } #[tokio::test] - async fn test_q2() -> Result<()> { + async fn test_q2() -> TestResult<()> { do_test(2).await } #[tokio::test] - async fn test_q3() -> Result<()> { + async fn test_q3() -> TestResult<()> { do_test(3).await } #[tokio::test] - async fn test_q4() -> Result<()> { + async fn test_q4() -> TestResult<()> { do_test(4).await } #[tokio::test] - async fn test_q5() -> Result<()> { + async fn test_q5() -> TestResult<()> { do_test(5).await } #[tokio::test] - async fn test_q6() -> Result<()> { + async fn test_q6() -> TestResult<()> { do_test(6).await } + #[ignore = "non-deterministic IN clause"] #[tokio::test] - async fn test_q7() -> Result<()> { + async fn test_q7() -> TestResult<()> { do_test(7).await } #[tokio::test] - async fn test_q8() -> Result<()> { + async fn test_q8() -> TestResult<()> { do_test(8).await } #[tokio::test] - async fn test_q9() -> Result<()> { + async fn test_q9() -> TestResult<()> { do_test(9).await } #[tokio::test] - async fn test_q10() -> Result<()> { + async fn test_q10() -> TestResult<()> { do_test(10).await } #[tokio::test] - async fn test_q11() -> Result<()> { + async fn test_q11() -> TestResult<()> { do_test(11).await } + #[ignore = "non-deterministic IN clause"] #[tokio::test] - async fn test_q12() -> Result<()> { + async fn test_q12() -> TestResult<()> { do_test(12).await } #[tokio::test] - async fn test_q13() -> Result<()> { + async fn test_q13() -> TestResult<()> { do_test(13).await } #[tokio::test] - async fn test_q14() -> Result<()> { + async fn test_q14() -> TestResult<()> { do_test(14).await } #[ignore] #[tokio::test] - async fn test_q15() -> Result<()> { + async fn test_q15() -> TestResult<()> { do_test(15).await } + // This test is ignored because there is some non-determinism + // in a part of the plan, see + // https://github.com/edmondop/datafusion-ray/actions/runs/11180062292/job/31080996808" + #[ignore = "non-deterministic IN clause"] #[tokio::test] - async fn test_q16() -> Result<()> { + async fn test_q16() -> TestResult<()> { do_test(16).await } #[tokio::test] - async fn test_q17() -> Result<()> { + async fn test_q17() -> TestResult<()> { do_test(17).await } #[tokio::test] - async fn test_q18() -> Result<()> { + async fn test_q18() -> TestResult<()> { do_test(18).await } + #[ignore = "non-deterministic IN clause"] #[tokio::test] - async fn test_q19() -> Result<()> { + async fn test_q19() -> TestResult<()> { do_test(19).await } #[tokio::test] - async fn test_q20() -> Result<()> { + async fn test_q20() -> TestResult<()> { do_test(20).await } #[tokio::test] - async fn test_q21() -> Result<()> { + async fn test_q21() -> TestResult<()> { do_test(21).await } #[tokio::test] - async fn test_q22() -> Result<()> { + async fn test_q22() -> TestResult<()> { do_test(22).await } - async fn do_test(n: u8) -> Result<()> { - let data_path = "/mnt/bigdata/tpch/sf10-parquet"; - if !Path::new(&data_path).exists() { - return Ok(()); - } + async fn do_test(n: u8) -> TestResult<()> { + let data_path = env::var("TPCH_DATA_PATH")?; let file = format!("testdata/queries/q{n}.sql"); let sql = fs::read_to_string(&file)?; - let config = SessionConfig::new().with_target_partitions(4); + let config = SessionConfig::new().with_target_partitions(1); let ctx = SessionContext::with_config(config); let tables = &[ "customer", "lineitem", "nation", "orders", "part", "partsupp", "region", "supplier", @@ -430,7 +437,14 @@ mod test { fs::write(&expected_file, &output)?; } let expected_plan = fs::read_to_string(&expected_file)?; - assert_eq!(expected_plan, output); + + let re = Regex::new(r":[^]]*]")?; + + // Remove the byte offsets from the plans, seems non repeatable + // between CI/CD and local + let cleaned_expected_plan = re.replace_all(&expected_plan, "]"); + let cleaned_output = re.replace_all(&output, "]"); + assert_eq!(cleaned_expected_plan, cleaned_output); Ok(()) } } diff --git a/testdata/expected-plans/q1.txt b/testdata/expected-plans/q1.txt index 2396d76..f37ef3a 100644 --- a/testdata/expected-plans/q1.txt +++ b/testdata/expected-plans/q1.txt @@ -2,47 +2,32 @@ DataFusion Logical Plan ======================= Sort: lineitem.l_returnflag ASC NULLS LAST, lineitem.l_linestatus ASC NULLS LAST - Projection: lineitem.l_returnflag, lineitem.l_linestatus, SUM(lineitem.l_quantity) AS sum_qty, SUM(lineitem.l_extendedprice) AS sum_base_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS sum_disc_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax) AS sum_charge, AVG(lineitem.l_quantity) AS avg_qty, AVG(lineitem.l_extendedprice) AS avg_price, AVG(lineitem.l_discount) AS avg_disc, COUNT(UInt8(1)) AS count_order - Aggregate: groupBy=[[lineitem.l_returnflag, lineitem.l_linestatus]], aggr=[[SUM(lineitem.l_quantity), SUM(lineitem.l_extendedprice), SUM(CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2))CAST(lineitem.l_discount AS Decimal128(23, 2))lineitem.l_discountDecimal128(Some(100),23,2)CAST(lineitem.l_extendedprice AS Decimal128(35, 4))lineitem.l_extendedprice AS lineitem.l_extendedprice * Decimal128(Some(100),23,2) - lineitem.l_discount) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), SUM(CAST(CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2))CAST(lineitem.l_discount AS Decimal128(23, 2))lineitem.l_discountDecimal128(Some(100),23,2)CAST(lineitem.l_extendedprice AS Decimal128(35, 4))lineitem.l_extendedprice AS lineitem.l_extendedprice * Decimal128(Some(100),23,2) - lineitem.l_discount AS Decimal128(38, 6)) * CAST(Decimal128(Some(100),23,2) + CAST(lineitem.l_tax AS Decimal128(23, 2)) AS Decimal128(38, 6))) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(UInt8(1))]] - Projection: CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4)) AS CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2))CAST(lineitem.l_discount AS Decimal128(23, 2))lineitem.l_discountDecimal128(Some(100),23,2)CAST(lineitem.l_extendedprice AS Decimal128(35, 4))lineitem.l_extendedprice, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus - Filter: lineitem.l_shipdate <= Date32("10493") - TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], partial_filters=[lineitem.l_shipdate <= Date32("10493")] + Projection: lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity) AS sum_qty, sum(lineitem.l_extendedprice) AS sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax) AS sum_charge, avg(lineitem.l_quantity) AS avg_qty, avg(lineitem.l_extendedprice) AS avg_price, avg(lineitem.l_discount) AS avg_disc, count(*) AS count_order + Aggregate: groupBy=[[lineitem.l_returnflag, lineitem.l_linestatus]], aggr=[[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(__common_expr_1) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(__common_expr_1 * (Decimal128(Some(1),20,0) + lineitem.l_tax)) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(Int64(1)) AS count(*)]] + Projection: lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS __common_expr_1, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus + Filter: lineitem.l_shipdate <= Date32("1998-09-24") + TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], partial_filters=[lineitem.l_shipdate <= Date32("1998-09-24")] DataFusion Physical Plan ======================== -SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] - SortExec: expr=[l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] - ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, SUM(lineitem.l_quantity)@2 as sum_qty, SUM(lineitem.l_extendedprice)@3 as sum_base_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, AVG(lineitem.l_quantity)@6 as avg_qty, AVG(lineitem.l_extendedprice)@7 as avg_price, AVG(lineitem.l_discount)@8 as avg_disc, COUNT(UInt8(1))@9 as count_order] - AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[SUM(lineitem.l_quantity), SUM(lineitem.l_extendedprice), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(UInt8(1))] +SortExec: expr=[l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[false] + ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, sum(lineitem.l_quantity)@2 as sum_qty, sum(lineitem.l_extendedprice)@3 as sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, avg(lineitem.l_quantity)@6 as avg_qty, avg(lineitem.l_extendedprice)@7 as avg_price, avg(lineitem.l_discount)@8 as avg_disc, count(*)@9 as count_order] + AggregateExec: mode=Single, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(*)] + ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_2, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[SUM(lineitem.l_quantity), SUM(lineitem.l_extendedprice), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(UInt8(1))] - ProjectionExec: expr=[CAST(l_extendedprice@1 AS Decimal128(35, 4)) * CAST(Some(100),23,2 - CAST(l_discount@2 AS Decimal128(23, 2)) AS Decimal128(35, 4)) as CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2))CAST(lineitem.l_discount AS Decimal128(23, 2))lineitem.l_discountDecimal128(Some(100),23,2)CAST(lineitem.l_extendedprice AS Decimal128(35, 4))lineitem.l_extendedprice, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_shipdate@6 <= 10493 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate@10 <= 10493, pruning_predicate=l_shipdate_min@0 <= 10493, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate] + FilterExec: l_shipdate@6 <= 1998-09-24 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], predicate=l_shipdate@10 <= 1998-09-24, pruning_predicate=CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_min@0 <= 1998-09-24 END, required_guarantees=[] RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 4)) - AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[SUM(lineitem.l_quantity), SUM(lineitem.l_extendedprice), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(UInt8(1))] - ProjectionExec: expr=[CAST(l_extendedprice@1 AS Decimal128(35, 4)) * CAST(Some(100),23,2 - CAST(l_discount@2 AS Decimal128(23, 2)) AS Decimal128(35, 4)) as CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2))CAST(lineitem.l_discount AS Decimal128(23, 2))lineitem.l_discountDecimal128(Some(100),23,2)CAST(lineitem.l_extendedprice AS Decimal128(35, 4))lineitem.l_extendedprice, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_shipdate@6 <= 10493 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate@10 <= 10493, pruning_predicate=l_shipdate_min@0 <= 10493, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate] - -Query Stage #1 (4 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 4)) - SortExec: expr=[l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] - ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, SUM(lineitem.l_quantity)@2 as sum_qty, SUM(lineitem.l_extendedprice)@3 as sum_base_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, AVG(lineitem.l_quantity)@6 as avg_qty, AVG(lineitem.l_extendedprice)@7 as avg_price, AVG(lineitem.l_discount)@8 as avg_disc, COUNT(UInt8(1))@9 as count_order] - AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[SUM(lineitem.l_quantity), SUM(lineitem.l_extendedprice), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(UInt8(1))] +Query Stage #0 (1 -> 1): +SortExec: expr=[l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[false] + ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, sum(lineitem.l_quantity)@2 as sum_qty, sum(lineitem.l_extendedprice)@3 as sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, avg(lineitem.l_quantity)@6 as avg_qty, avg(lineitem.l_extendedprice)@7 as avg_price, avg(lineitem.l_discount)@8 as avg_disc, count(*)@9 as count_order] + AggregateExec: mode=Single, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(*)] + ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_2, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 4)) - -Query Stage #2 (4 -> 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 }], 4)) + FilterExec: l_shipdate@6 <= 1998-09-24 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], predicate=l_shipdate@10 <= 1998-09-24, pruning_predicate=CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_min@0 <= 1998-09-24 END, required_guarantees=[] diff --git a/testdata/expected-plans/q10.txt b/testdata/expected-plans/q10.txt index 039c96f..e9e4c67 100644 --- a/testdata/expected-plans/q10.txt +++ b/testdata/expected-plans/q10.txt @@ -3,8 +3,8 @@ DataFusion Logical Plan Limit: skip=0, fetch=20 Sort: revenue DESC NULLS FIRST, fetch=20 - Projection: customer.c_custkey, customer.c_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue, customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment - Aggregate: groupBy=[[customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment]], aggr=[[SUM(CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] + Projection: customer.c_custkey, customer.c_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue, customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment + Aggregate: groupBy=[[customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment]], aggr=[[sum(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] Projection: customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name Inner Join: customer.c_nationkey = nation.n_nationkey Projection: customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount @@ -13,8 +13,8 @@ Limit: skip=0, fetch=20 Inner Join: customer.c_custkey = orders.o_custkey TableScan: customer projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment] Projection: orders.o_orderkey, orders.o_custkey - Filter: orders.o_orderdate >= Date32("8582") AND orders.o_orderdate < Date32("8674") - TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("8582"), orders.o_orderdate < Date32("8674")] + Filter: orders.o_orderdate >= Date32("1993-07-01") AND orders.o_orderdate < Date32("1993-10-01") + TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("1993-07-01"), orders.o_orderdate < Date32("1993-10-01")] Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount Filter: lineitem.l_returnflag = Utf8("R") TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], partial_filters=[lineitem.l_returnflag = Utf8("R")] @@ -23,112 +23,49 @@ Limit: skip=0, fetch=20 DataFusion Physical Plan ======================== -GlobalLimitExec: skip=0, fetch=20 - SortPreservingMergeExec: [revenue@2 DESC] - SortExec: fetch=20, expr=[revenue@2 DESC] - ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment] - AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: 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 }], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@7 as l_extendedprice, l_discount@8 as l_discount, n_name@10 as n_name] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_nationkey", index: 3 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "c_nationkey", index: 3 }], 4), input_partitions=4 - ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_nationkey@3 as c_nationkey, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@9 as l_extendedprice, l_discount@10 as l_discount] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 7 }, Column { name: "l_orderkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 7 }], 4), input_partitions=4 - ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_nationkey@3 as c_nationkey, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, o_orderkey@7 as o_orderkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4), input_partitions=4 - ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: o_orderdate@2 >= 8582 AND o_orderdate@2 < 8674 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderdate@4 >= 8582 AND o_orderdate@4 < 8674, pruning_predicate=o_orderdate_max@0 >= 8582 AND o_orderdate_min@1 < 8674, projection=[o_orderkey, o_custkey, o_orderdate] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_returnflag@3 = R - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_returnflag@8 = R, pruning_predicate=l_returnflag_min@0 <= R AND R <= l_returnflag_max@1, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag] +SortExec: TopK(fetch=20), expr=[revenue@2 DESC], preserve_partitioning=[false] + ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment] + AggregateExec: mode=Single, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + ProjectionExec: expr=[c_custkey@1 as c_custkey, c_name@2 as c_name, c_address@3 as c_address, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@7 as l_extendedprice, l_discount@8 as l_discount, n_name@0 as n_name] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@3)], projection=[n_name@1, c_custkey@2, c_name@3, c_address@4, c_phone@6, c_acctbal@7, c_comment@8, l_extendedprice@9, l_discount@10] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, projection=[n_nationkey, n_name] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/customer.parquet:0..13211178]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment] + ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=1 - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_name] + FilterExec: o_orderdate@2 >= 1993-07-01 AND o_orderdate@2 < 1993-10-01 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_orderkey, o_custkey, o_orderdate], predicate=o_orderdate@4 >= 1993-07-01 AND o_orderdate@4 < 1993-10-01, pruning_predicate=CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_max@0 >= 1993-07-01 END AND CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_min@3 < 1993-10-01 END, required_guarantees=[] + ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: l_returnflag@3 = R + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], predicate=l_returnflag@8 = R, pruning_predicate=CASE WHEN l_returnflag_null_count@2 = l_returnflag_row_count@3 THEN false ELSE l_returnflag_min@0 <= R AND R <= l_returnflag_max@1 END, required_guarantees=[l_returnflag in (R)] RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment] - -Query Stage #1 (4 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) - ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: o_orderdate@2 >= 8582 AND o_orderdate@2 < 8674 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderdate@4 >= 8582 AND o_orderdate@4 < 8674, pruning_predicate=o_orderdate_max@0 >= 8582 AND o_orderdate_min@1 < 8674, projection=[o_orderkey, o_custkey, o_orderdate] - -Query Stage #2 (4 -> 4): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_orderkey", index: 7 }], 4)) - ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_nationkey@3 as c_nationkey, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, o_orderkey@7 as o_orderkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) - -Query Stage #3 (4 -> 4): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_returnflag@3 = R - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_returnflag@8 = R, pruning_predicate=l_returnflag_min@0 <= R AND R <= l_returnflag_max@1, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag] - -Query Stage #4 (4 -> 4): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "c_nationkey", index: 3 }], 4)) - ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_nationkey@3 as c_nationkey, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@9 as l_extendedprice, l_discount@10 as l_discount] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 7 }, Column { name: "l_orderkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_orderkey", index: 7 }], 4)) +Query Stage #0 (1 -> 1): +SortExec: TopK(fetch=20), expr=[revenue@2 DESC], preserve_partitioning=[false] + ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment] + AggregateExec: mode=Single, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + ProjectionExec: expr=[c_custkey@1 as c_custkey, c_name@2 as c_name, c_address@3 as c_address, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@7 as l_extendedprice, l_discount@8 as l_discount, n_name@0 as n_name] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - -Query Stage #5 (1 -> 4): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_name] - -Query Stage #6 (4 -> 4): -ShuffleWriterExec(stage_id=6, output_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 }], 4)) - AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@7 as l_extendedprice, l_discount@8 as l_discount, n_name@10 as n_name] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_nationkey", index: 3 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "c_nationkey", index: 3 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - -Query Stage #7 (4 -> 4): -ShuffleWriterExec(stage_id=7, output_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 }], 4)) - SortExec: fetch=20, expr=[revenue@2 DESC] - ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment] - AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - 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 }], 4)) - -Query Stage #8 (1 -> 1): -GlobalLimitExec: skip=0, fetch=20 - SortPreservingMergeExec: [revenue@2 DESC] - 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 }], 4)) + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@3)], projection=[n_name@1, c_custkey@2, c_name@3, c_address@4, c_phone@6, c_acctbal@7, c_comment@8, l_extendedprice@9, l_discount@10] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, projection=[n_nationkey, n_name] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/customer.parquet:0..13211178]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment] + ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: o_orderdate@2 >= 1993-07-01 AND o_orderdate@2 < 1993-10-01 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_orderkey, o_custkey, o_orderdate], predicate=o_orderdate@4 >= 1993-07-01 AND o_orderdate@4 < 1993-10-01, pruning_predicate=CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_max@0 >= 1993-07-01 END AND CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_min@3 < 1993-10-01 END, required_guarantees=[] + ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: l_returnflag@3 = R + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], predicate=l_returnflag@8 = R, pruning_predicate=CASE WHEN l_returnflag_null_count@2 = l_returnflag_row_count@3 THEN false ELSE l_returnflag_min@0 <= R AND R <= l_returnflag_max@1 END, required_guarantees=[l_returnflag in (R)] diff --git a/testdata/expected-plans/q11.txt b/testdata/expected-plans/q11.txt index 98c619a..8da9394 100644 --- a/testdata/expected-plans/q11.txt +++ b/testdata/expected-plans/q11.txt @@ -2,187 +2,93 @@ DataFusion Logical Plan ======================= Sort: value DESC NULLS FIRST - Projection: partsupp.ps_partkey, SUM(partsupp.ps_supplycost * partsupp.ps_availqty) AS value - Filter: CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty) AS Decimal128(38, 15)) > CAST(__scalar_sq_1.__value AS Decimal128(38, 15)) - CrossJoin: - Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[SUM(CAST(partsupp.ps_supplycost AS Decimal128(22, 2)) * CAST(partsupp.ps_availqty AS Decimal128(22, 2)))]] - Projection: partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost - Inner Join: supplier.s_nationkey = nation.n_nationkey - Projection: partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey - Inner Join: partsupp.ps_suppkey = supplier.s_suppkey - TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost] - TableScan: supplier projection=[s_suppkey, s_nationkey] - Projection: nation.n_nationkey - Filter: nation.n_name = Utf8("ALGERIA") - TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("ALGERIA")] - SubqueryAlias: __scalar_sq_1 - Projection: CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty) AS Float64) * Float64(0.0001) AS __value - Aggregate: groupBy=[[]], aggr=[[SUM(CAST(partsupp.ps_supplycost AS Decimal128(22, 2)) * CAST(partsupp.ps_availqty AS Decimal128(22, 2)))]] - Projection: partsupp.ps_availqty, partsupp.ps_supplycost - Inner Join: supplier.s_nationkey = nation.n_nationkey - Projection: partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey - Inner Join: partsupp.ps_suppkey = supplier.s_suppkey - TableScan: partsupp projection=[ps_suppkey, ps_availqty, ps_supplycost] - TableScan: supplier projection=[s_suppkey, s_nationkey] - Projection: nation.n_nationkey - Filter: nation.n_name = Utf8("ALGERIA") - TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("ALGERIA")] + Projection: partsupp.ps_partkey, sum(partsupp.ps_supplycost * partsupp.ps_availqty) AS value + Inner Join: Filter: CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty) AS Decimal128(38, 15)) > __scalar_sq_1.sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001) + Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[sum(partsupp.ps_supplycost * CAST(partsupp.ps_availqty AS Decimal128(10, 0)))]] + Projection: partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost + Inner Join: supplier.s_nationkey = nation.n_nationkey + Projection: partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey + Inner Join: partsupp.ps_suppkey = supplier.s_suppkey + TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost] + TableScan: supplier projection=[s_suppkey, s_nationkey] + Projection: nation.n_nationkey + Filter: nation.n_name = Utf8("ALGERIA") + TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("ALGERIA")] + SubqueryAlias: __scalar_sq_1 + Projection: CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty) AS Float64) * Float64(0.0001) AS Decimal128(38, 15)) + Aggregate: groupBy=[[]], aggr=[[sum(partsupp.ps_supplycost * CAST(partsupp.ps_availqty AS Decimal128(10, 0)))]] + Projection: partsupp.ps_availqty, partsupp.ps_supplycost + Inner Join: supplier.s_nationkey = nation.n_nationkey + Projection: partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey + Inner Join: partsupp.ps_suppkey = supplier.s_suppkey + TableScan: partsupp projection=[ps_suppkey, ps_availqty, ps_supplycost] + TableScan: supplier projection=[s_suppkey, s_nationkey] + Projection: nation.n_nationkey + Filter: nation.n_name = Utf8("ALGERIA") + TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("ALGERIA")] DataFusion Physical Plan ======================== -SortExec: expr=[value@1 DESC] - ProjectionExec: expr=[ps_partkey@0 as ps_partkey, SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as value] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@1 AS Decimal128(38, 15)) > CAST(__value@2 AS Decimal128(38, 15)) - CrossJoinExec - CoalescePartitionsExec - AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] +SortExec: expr=[value@1 DESC], preserve_partitioning=[false] + ProjectionExec: expr=[ps_partkey@1 as ps_partkey, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@2 as value] + NestedLoopJoinExec: join_type=Inner, filter=CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Decimal128(38, 15)) > sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@1 + ProjectionExec: expr=[CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] + AggregateExec: mode=Single, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[ps_availqty@1, ps_supplycost@2] + ProjectionExec: expr=[n_nationkey@0 as n_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: n_name@1 = ALGERIA + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, 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)] + ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_nationkey@1, ps_availqty@3, ps_supplycost@4] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_nationkey] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/partsupp.parquet:0..41091907]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost] + AggregateExec: mode=Single, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[ps_partkey@1, ps_availqty@2, ps_supplycost@3] + ProjectionExec: expr=[n_nationkey@0 as n_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: n_name@1 = ALGERIA + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, 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)] + ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_availqty@2 as ps_availqty, ps_supplycost@3 as ps_supplycost, s_nationkey@0 as s_nationkey] CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] - ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 3 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 4), input_partitions=4 - ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_availqty@2 as ps_availqty, ps_supplycost@3 as ps_supplycost, s_nationkey@5 as s_nationkey] - ProjectionExec: expr=[ps_partkey@2 as ps_partkey, ps_suppkey@3 as ps_suppkey, ps_availqty@4 as ps_availqty, ps_supplycost@5 as ps_supplycost, s_suppkey@0 as s_suppkey, s_nationkey@1 as s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "ps_suppkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4 - RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - ProjectionExec: expr=[n_nationkey@0 as n_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: n_name@1 = ALGERIA - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, predicate=n_name@1 = ALGERIA, pruning_predicate=n_name_min@0 <= ALGERIA AND ALGERIA <= n_name_max@1, projection=[n_nationkey, n_name] - ProjectionExec: expr=[CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 as __value] - AggregateExec: mode=Final, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] - CoalescePartitionsExec - AggregateExec: mode=Partial, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] - ProjectionExec: expr=[ps_availqty@0 as ps_availqty, ps_supplycost@1 as ps_supplycost] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 2 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 4), input_partitions=4 - ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@4 as s_nationkey] - ProjectionExec: expr=[ps_suppkey@2 as ps_suppkey, ps_availqty@3 as ps_availqty, ps_supplycost@4 as ps_supplycost, s_suppkey@0 as s_suppkey, s_nationkey@1 as s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "ps_suppkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4 - RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - ProjectionExec: expr=[n_nationkey@0 as n_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: n_name@1 = ALGERIA - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, predicate=n_name@1 = ALGERIA, pruning_predicate=n_name_min@0 <= ALGERIA AND ALGERIA <= n_name_max@1, projection=[n_nationkey, n_name] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@1)], projection=[s_nationkey@1, ps_partkey@2, ps_availqty@4, ps_supplycost@5] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_nationkey] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/partsupp.parquet:0..41091907]]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost] RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] - -Query Stage #1 (4 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost] - -Query Stage #2 (4 -> 4): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 4)) - ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_availqty@2 as ps_availqty, ps_supplycost@3 as ps_supplycost, s_nationkey@5 as s_nationkey] - ProjectionExec: expr=[ps_partkey@2 as ps_partkey, ps_suppkey@3 as ps_suppkey, ps_availqty@4 as ps_availqty, ps_supplycost@5 as ps_supplycost, s_suppkey@0 as s_suppkey, s_nationkey@1 as s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "ps_suppkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 4)) - -Query Stage #3 (1 -> 4): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - ProjectionExec: expr=[n_nationkey@0 as n_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: n_name@1 = ALGERIA - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, predicate=n_name@1 = ALGERIA, pruning_predicate=n_name_min@0 <= ALGERIA AND ALGERIA <= n_name_max@1, projection=[n_nationkey, n_name] - -Query Stage #4 (4 -> 4): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) - AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] - ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 3 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - -Query Stage #5 (4 -> 4): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) - AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) - -Query Stage #6 (4 -> 4): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] - -Query Stage #7 (4 -> 4): -ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost] - -Query Stage #8 (4 -> 4): -ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 4)) - ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@4 as s_nationkey] - ProjectionExec: expr=[ps_suppkey@2 as ps_suppkey, ps_availqty@3 as ps_availqty, ps_supplycost@4 as ps_supplycost, s_suppkey@0 as s_suppkey, s_nationkey@1 as s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "ps_suppkey", index: 0 })] +Query Stage #0 (1 -> 1): +SortExec: expr=[value@1 DESC], preserve_partitioning=[false] + ProjectionExec: expr=[ps_partkey@1 as ps_partkey, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@2 as value] + NestedLoopJoinExec: join_type=Inner, filter=CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Decimal128(38, 15)) > sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@1 + ProjectionExec: expr=[CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] + AggregateExec: mode=Single, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 4)) - -Query Stage #9 (1 -> 4): -ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - ProjectionExec: expr=[n_nationkey@0 as n_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: n_name@1 = ALGERIA - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, predicate=n_name@1 = ALGERIA, pruning_predicate=n_name_min@0 <= ALGERIA AND ALGERIA <= n_name_max@1, projection=[n_nationkey, n_name] - -Query Stage #10 (4 -> 4): -ShuffleWriterExec(stage_id=10, output_partitioning=Hash([], 4)) - AggregateExec: mode=Partial, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] - ProjectionExec: expr=[ps_availqty@0 as ps_availqty, ps_supplycost@1 as ps_supplycost] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 2 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - -Query Stage #11 (1 -> 1): -SortExec: expr=[value@1 DESC] - ProjectionExec: expr=[ps_partkey@0 as ps_partkey, SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as value] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@1 AS Decimal128(38, 15)) > CAST(__value@2 AS Decimal128(38, 15)) - CrossJoinExec - CoalescePartitionsExec - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) - ProjectionExec: expr=[CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 as __value] - AggregateExec: mode=Final, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] - CoalescePartitionsExec - ShuffleReaderExec(stage_id=10, input_partitioning=Hash([], 4)) + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[ps_availqty@1, ps_supplycost@2] + ProjectionExec: expr=[n_nationkey@0 as n_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: n_name@1 = ALGERIA + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, 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)] + ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_nationkey@1, ps_availqty@3, ps_supplycost@4] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_nationkey] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/partsupp.parquet:0..41091907]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost] + AggregateExec: mode=Single, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[ps_partkey@1, ps_availqty@2, ps_supplycost@3] + ProjectionExec: expr=[n_nationkey@0 as n_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: n_name@1 = ALGERIA + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, 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)] + ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_availqty@2 as ps_availqty, ps_supplycost@3 as ps_supplycost, s_nationkey@0 as s_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@1)], projection=[s_nationkey@1, ps_partkey@2, ps_availqty@4, ps_supplycost@5] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_nationkey] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/partsupp.parquet:0..41091907]]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost] diff --git a/testdata/expected-plans/q12.txt b/testdata/expected-plans/q12.txt index 2ba8687..cf5a5bd 100644 --- a/testdata/expected-plans/q12.txt +++ b/testdata/expected-plans/q12.txt @@ -2,72 +2,43 @@ 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: 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("SHIP") OR lineitem.l_shipmode = Utf8("FOB")) AND lineitem.l_commitdate < lineitem.l_receiptdate AND lineitem.l_shipdate < lineitem.l_commitdate AND lineitem.l_receiptdate >= Date32("9131") AND lineitem.l_receiptdate < Date32("9496") - TableScan: lineitem projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], partial_filters=[lineitem.l_shipmode = Utf8("SHIP") OR lineitem.l_shipmode = Utf8("FOB"), lineitem.l_commitdate < lineitem.l_receiptdate, lineitem.l_shipdate < lineitem.l_commitdate, lineitem.l_receiptdate >= Date32("9131"), lineitem.l_receiptdate < Date32("9496")] + 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] - 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)] +SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[false] + 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=Single, 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 - RepartitionExec: partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 4), input_partitions=4 - 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@3 as l_shipmode] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 0 }, Column { name: "l_orderkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, projection=[o_orderkey, o_orderpriority] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: (l_shipmode@4 = SHIP OR l_shipmode@4 = FOB) AND l_commitdate@2 < l_receiptdate@3 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 9131 AND l_receiptdate@3 < 9496 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=(l_shipmode@14 = SHIP OR l_shipmode@14 = FOB) AND l_commitdate@11 < l_receiptdate@12 AND l_shipdate@10 < l_commitdate@11 AND l_receiptdate@12 >= 9131 AND l_receiptdate@12 < 9496, pruning_predicate=(l_shipmode_min@0 <= SHIP AND SHIP <= l_shipmode_max@1 OR l_shipmode_min@0 <= FOB AND FOB <= l_shipmode_max@1) AND l_receiptdate_max@2 >= 9131 AND l_receiptdate_min@3 < 9496, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] + ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] + 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 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, 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)] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_orderkey, o_orderpriority] RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, projection=[o_orderkey, o_orderpriority] - -Query Stage #1 (4 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: (l_shipmode@4 = SHIP OR l_shipmode@4 = FOB) AND l_commitdate@2 < l_receiptdate@3 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 9131 AND l_receiptdate@3 < 9496 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=(l_shipmode@14 = SHIP OR l_shipmode@14 = FOB) AND l_commitdate@11 < l_receiptdate@12 AND l_shipdate@10 < l_commitdate@11 AND l_receiptdate@12 >= 9131 AND l_receiptdate@12 < 9496, pruning_predicate=(l_shipmode_min@0 <= SHIP AND SHIP <= l_shipmode_max@1 OR l_shipmode_min@0 <= FOB AND FOB <= l_shipmode_max@1) AND l_receiptdate_max@2 >= 9131 AND l_receiptdate_min@3 < 9496, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode] - -Query Stage #2 (4 -> 4): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 4)) - 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@3 as l_shipmode] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 0 }, Column { name: "l_orderkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - -Query Stage #3 (4 -> 4): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 4)) - SortExec: expr=[l_shipmode@0 ASC NULLS LAST] - 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)] +Query Stage #0 (1 -> 1): +SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[false] + 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=Single, 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 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 4)) - -Query Stage #4 (4 -> 1): -SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 4)) + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] + ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] + 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 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, 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)] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_orderkey, o_orderpriority] diff --git a/testdata/expected-plans/q13.txt b/testdata/expected-plans/q13.txt index 9d2d3ab..99712a4 100644 --- a/testdata/expected-plans/q13.txt +++ b/testdata/expected-plans/q13.txt @@ -1,85 +1,49 @@ DataFusion Logical Plan ======================= -Sort: custdist DESC NULLS FIRST, c_count DESC NULLS FIRST - Projection: c_count, COUNT(UInt8(1)) AS custdist - Aggregate: groupBy=[[c_count]], aggr=[[COUNT(UInt8(1))]] - Projection: c_orders.COUNT(orders.o_orderkey) AS c_count - SubqueryAlias: c_orders - Projection: COUNT(orders.o_orderkey) - Aggregate: groupBy=[[customer.c_custkey]], aggr=[[COUNT(orders.o_orderkey)]] - Projection: customer.c_custkey, orders.o_orderkey - Left Join: customer.c_custkey = orders.o_custkey - TableScan: customer projection=[c_custkey] - Projection: orders.o_orderkey, orders.o_custkey - Filter: orders.o_comment NOT LIKE Utf8("%express%requests%") - TableScan: orders projection=[o_orderkey, o_custkey, o_comment], partial_filters=[orders.o_comment NOT LIKE Utf8("%express%requests%")] +Sort: custdist DESC NULLS FIRST, c_orders.c_count DESC NULLS FIRST + Projection: c_orders.c_count, count(*) AS custdist + Aggregate: groupBy=[[c_orders.c_count]], aggr=[[count(Int64(1)) AS count(*)]] + SubqueryAlias: c_orders + Projection: count(orders.o_orderkey) AS c_count + Aggregate: groupBy=[[customer.c_custkey]], aggr=[[count(orders.o_orderkey)]] + Projection: customer.c_custkey, orders.o_orderkey + Left Join: customer.c_custkey = orders.o_custkey + TableScan: customer projection=[c_custkey] + Projection: orders.o_orderkey, orders.o_custkey + Filter: orders.o_comment NOT LIKE Utf8("%express%requests%") + TableScan: orders projection=[o_orderkey, o_custkey, o_comment], partial_filters=[orders.o_comment NOT LIKE Utf8("%express%requests%")] DataFusion Physical Plan ======================== -SortPreservingMergeExec: [custdist@1 DESC,c_count@0 DESC] - SortExec: expr=[custdist@1 DESC,c_count@0 DESC] - ProjectionExec: expr=[c_count@0 as c_count, COUNT(UInt8(1))@1 as custdist] - AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[COUNT(UInt8(1))] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "c_count", index: 0 }], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[COUNT(UInt8(1))] - ProjectionExec: expr=[COUNT(orders.o_orderkey)@0 as c_count] - ProjectionExec: expr=[COUNT(orders.o_orderkey)@1 as COUNT(orders.o_orderkey)] - AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)] - AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)] - ProjectionExec: expr=[c_custkey@0 as c_custkey, o_orderkey@1 as o_orderkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Left, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, projection=[c_custkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4), input_partitions=4 - ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: o_comment@2 NOT LIKE %express%requests% - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_comment@8 NOT LIKE %express%requests%, projection=[o_orderkey, o_custkey, o_comment] +SortExec: expr=[custdist@1 DESC,c_count@0 DESC], preserve_partitioning=[false] + ProjectionExec: expr=[c_count@0 as c_count, count(*)@1 as custdist] + AggregateExec: mode=Single, gby=[c_count@0 as c_count], aggr=[count(*)] + ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] + AggregateExec: mode=Single, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/customer.parquet:0..13211178]]}, projection=[c_custkey] + ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: o_comment@2 NOT LIKE %express%requests% + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_orderkey, o_custkey, o_comment], predicate=o_comment@8 NOT LIKE %express%requests% RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, projection=[c_custkey] - -Query Stage #1 (4 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) - ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: o_comment@2 NOT LIKE %express%requests% - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_comment@8 NOT LIKE %express%requests%, projection=[o_orderkey, o_custkey, o_comment] - -Query Stage #2 (4 -> 4): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "c_count", index: 0 }], 4)) - AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[COUNT(UInt8(1))] - ProjectionExec: expr=[COUNT(orders.o_orderkey)@0 as c_count] - ProjectionExec: expr=[COUNT(orders.o_orderkey)@1 as COUNT(orders.o_orderkey)] - AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)] - AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)] - ProjectionExec: expr=[c_custkey@0 as c_custkey, o_orderkey@1 as o_orderkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Left, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) - -Query Stage #3 (4 -> 4): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "c_count", index: 0 }], 4)) - SortExec: expr=[custdist@1 DESC,c_count@0 DESC] - ProjectionExec: expr=[c_count@0 as c_count, COUNT(UInt8(1))@1 as custdist] - AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[COUNT(UInt8(1))] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "c_count", index: 0 }], 4)) - -Query Stage #4 (4 -> 1): -SortPreservingMergeExec: [custdist@1 DESC,c_count@0 DESC] - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "c_count", index: 0 }], 4)) +Query Stage #0 (1 -> 1): +SortExec: expr=[custdist@1 DESC,c_count@0 DESC], preserve_partitioning=[false] + ProjectionExec: expr=[c_count@0 as c_count, count(*)@1 as custdist] + AggregateExec: mode=Single, gby=[c_count@0 as c_count], aggr=[count(*)] + ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] + AggregateExec: mode=Single, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/customer.parquet:0..13211178]]}, projection=[c_custkey] + ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: o_comment@2 NOT LIKE %express%requests% + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_orderkey, o_custkey, o_comment], predicate=o_comment@8 NOT LIKE %express%requests% diff --git a/testdata/expected-plans/q14.txt b/testdata/expected-plans/q14.txt index 54aaed3..a2c3d52 100644 --- a/testdata/expected-plans/q14.txt +++ b/testdata/expected-plans/q14.txt @@ -1,63 +1,41 @@ DataFusion Logical Plan ======================= -Projection: Float64(100) * CAST(SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END) AS Float64) / CAST(SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS Float64) AS promo_revenue - Aggregate: groupBy=[[]], aggr=[[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2))CAST(lineitem.l_discount AS Decimal128(23, 2))lineitem.l_discountDecimal128(Some(100),23,2)CAST(lineitem.l_extendedprice AS Decimal128(35, 4))lineitem.l_extendedprice AS lineitem.l_extendedprice * Decimal128(Some(100),23,2) - lineitem.l_discount ELSE Decimal128(Some(0),35,4) END) AS SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2))CAST(lineitem.l_discount AS Decimal128(23, 2))lineitem.l_discountDecimal128(Some(100),23,2)CAST(lineitem.l_extendedprice AS Decimal128(35, 4))lineitem.l_extendedprice AS lineitem.l_extendedprice * Decimal128(Some(100),23,2) - lineitem.l_discount) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] - Projection: CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4)) AS CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2))CAST(lineitem.l_discount AS Decimal128(23, 2))lineitem.l_discountDecimal128(Some(100),23,2)CAST(lineitem.l_extendedprice AS Decimal128(35, 4))lineitem.l_extendedprice, part.p_type +Projection: Float64(100) * CAST(sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END) AS Float64) / CAST(sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS Float64) AS promo_revenue + Aggregate: groupBy=[[]], aggr=[[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN __common_expr_1 ELSE Decimal128(Some(0),35,4) END) AS sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(__common_expr_1) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] + Projection: lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS __common_expr_1, part.p_type Inner Join: lineitem.l_partkey = part.p_partkey Projection: lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount - Filter: lineitem.l_shipdate >= Date32("9162") AND lineitem.l_shipdate < Date32("9190") - TableScan: lineitem projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9162"), lineitem.l_shipdate < Date32("9190")] + Filter: lineitem.l_shipdate >= Date32("1995-02-01") AND lineitem.l_shipdate < Date32("1995-03-01") + TableScan: lineitem projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("1995-02-01"), lineitem.l_shipdate < Date32("1995-03-01")] TableScan: part projection=[p_partkey, p_type] DataFusion Physical Plan ======================== -ProjectionExec: expr=[100 * CAST(SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END)@0 AS Float64) / CAST(SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 AS Float64) as promo_revenue] - AggregateExec: mode=Final, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - CoalescePartitionsExec - AggregateExec: mode=Partial, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - ProjectionExec: expr=[CAST(l_extendedprice@1 AS Decimal128(35, 4)) * CAST(Some(100),23,2 - CAST(l_discount@2 AS Decimal128(23, 2)) AS Decimal128(35, 4)) as CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2))CAST(lineitem.l_discount AS Decimal128(23, 2))lineitem.l_discountDecimal128(Some(100),23,2)CAST(lineitem.l_extendedprice AS Decimal128(35, 4))lineitem.l_extendedprice, p_type@4 as p_type] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_shipdate@3 >= 9162 AND l_shipdate@3 < 9190 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate@10 >= 9162 AND l_shipdate@10 < 9190, pruning_predicate=l_shipdate_max@0 >= 9162 AND l_shipdate_min@1 < 9190, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, projection=[p_partkey, p_type] +ProjectionExec: expr=[100 * CAST(sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END)@0 AS Float64) / CAST(sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 AS Float64) as promo_revenue] + AggregateExec: mode=Single, gby=[], aggr=[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_2, p_type@0 as p_type] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_type@1, l_extendedprice@3, l_discount@4] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/part.parquet]]}, projection=[p_partkey, p_type] + ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: l_shipdate@3 >= 1995-02-01 AND l_shipdate@3 < 1995-03-01 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], predicate=l_shipdate@10 >= 1995-02-01 AND l_shipdate@10 < 1995-03-01, pruning_predicate=CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_max@0 >= 1995-02-01 END AND CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_min@3 < 1995-03-01 END, required_guarantees=[] RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4)) - ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_shipdate@3 >= 9162 AND l_shipdate@3 < 9190 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate@10 >= 9162 AND l_shipdate@10 < 9190, pruning_predicate=l_shipdate_max@0 >= 9162 AND l_shipdate_min@1 < 9190, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate] - -Query Stage #1 (4 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, projection=[p_partkey, p_type] - -Query Stage #2 (4 -> 4): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([], 4)) - AggregateExec: mode=Partial, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - ProjectionExec: expr=[CAST(l_extendedprice@1 AS Decimal128(35, 4)) * CAST(Some(100),23,2 - CAST(l_discount@2 AS Decimal128(23, 2)) AS Decimal128(35, 4)) as CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2))CAST(lineitem.l_discount AS Decimal128(23, 2))lineitem.l_discountDecimal128(Some(100),23,2)CAST(lineitem.l_extendedprice AS Decimal128(35, 4))lineitem.l_extendedprice, p_type@4 as p_type] +Query Stage #0 (1 -> 1): +ProjectionExec: expr=[100 * CAST(sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END)@0 AS Float64) / CAST(sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 AS Float64) as promo_revenue] + AggregateExec: mode=Single, gby=[], aggr=[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_2, p_type@0 as p_type] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) - -Query Stage #3 (1 -> 1): -ProjectionExec: expr=[100 * CAST(SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END)@0 AS Float64) / CAST(SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 AS Float64) as promo_revenue] - AggregateExec: mode=Final, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - CoalescePartitionsExec - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([], 4)) + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_type@1, l_extendedprice@3, l_discount@4] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/part.parquet]]}, projection=[p_partkey, p_type] + ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: l_shipdate@3 >= 1995-02-01 AND l_shipdate@3 < 1995-03-01 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], predicate=l_shipdate@10 >= 1995-02-01 AND l_shipdate@10 < 1995-03-01, pruning_predicate=CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_max@0 >= 1995-02-01 END AND CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_min@3 < 1995-03-01 END, required_guarantees=[] diff --git a/testdata/expected-plans/q16.txt b/testdata/expected-plans/q16.txt index d40a22c..fdd9b0b 100644 --- a/testdata/expected-plans/q16.txt +++ b/testdata/expected-plans/q16.txt @@ -2,9 +2,9 @@ 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: group_alias_0 AS part.p_brand, group_alias_1 AS part.p_type, group_alias_2 AS part.p_size, COUNT(alias1) AS supplier_cnt - Aggregate: groupBy=[[group_alias_0, group_alias_1, group_alias_2]], aggr=[[COUNT(alias1)]] - Aggregate: groupBy=[[part.p_brand AS group_alias_0, part.p_type AS group_alias_1, part.p_size AS group_alias_2, partsupp.ps_suppkey AS alias1]], aggr=[[]] + 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 @@ -12,102 +12,50 @@ Sort: supplier_cnt DESC NULLS FIRST, part.p_brand ASC NULLS LAST, part.p_type AS 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 AS s_suppkey + 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] - ProjectionExec: expr=[group_alias_0@0 as part.p_brand, group_alias_1@1 as part.p_type, group_alias_2@2 as part.p_size, COUNT(alias1)@3 as supplier_cnt] - AggregateExec: mode=FinalPartitioned, gby=[group_alias_0@0 as group_alias_0, group_alias_1@1 as group_alias_1, group_alias_2@2 as group_alias_2], aggr=[COUNT(alias1)] +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=[false] + 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=Single, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] + AggregateExec: mode=Single, 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 - RepartitionExec: partitioning=Hash([Column { name: "group_alias_0", index: 0 }, Column { name: "group_alias_1", index: 1 }, Column { name: "group_alias_2", index: 2 }], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[group_alias_0@0 as group_alias_0, group_alias_1@1 as group_alias_1, group_alias_2@2 as group_alias_2], aggr=[COUNT(alias1)] - AggregateExec: mode=FinalPartitioned, gby=[group_alias_0@0 as group_alias_0, group_alias_1@1 as group_alias_1, group_alias_2@2 as group_alias_2, alias1@3 as alias1], aggr=[] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "group_alias_0", index: 0 }, Column { name: "group_alias_1", index: 1 }, Column { name: "group_alias_2", index: 2 }, Column { name: "alias1", index: 3 }], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[p_brand@1 as group_alias_0, p_type@2 as group_alias_1, p_size@3 as group_alias_2, ps_suppkey@0 as alias1], aggr=[] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(Column { name: "ps_suppkey", index: 0 }, Column { name: "s_suppkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[ps_suppkey@1 as ps_suppkey, p_brand@3 as p_brand, p_type@4 as p_type, p_size@5 as p_size] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "ps_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_partkey, ps_suppkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4), input_partitions=4 - 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: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_brand@3 != Brand#14 AND p_type@4 NOT LIKE SMALL PLATED% AND Use p_size@5 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) }]), pruning_predicate=(p_brand_min@0 != Brand#14 OR Brand#14 != p_brand_max@1) AND (p_size_min@2 <= 14 AND 14 <= p_size_max@3 OR p_size_min@2 <= 6 AND 6 <= p_size_max@3 OR p_size_min@2 <= 5 AND 5 <= p_size_max@3 OR p_size_min@2 <= 31 AND 31 <= p_size_max@3 OR p_size_min@2 <= 49 AND 49 <= p_size_max@3 OR p_size_min@2 <= 15 AND 15 <= p_size_max@3 OR p_size_min@2 <= 41 AND 41 <= p_size_max@3 OR p_size_min@2 <= 47 AND 47 <= p_size_max@3), projection=[p_partkey, p_brand, p_type, p_size] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[s_suppkey@0 as s_suppkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: s_comment@1 LIKE %Customer%Complaints% - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, predicate=s_comment@6 LIKE %Customer%Complaints%, projection=[s_suppkey, s_comment] + HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(s_suppkey@0, ps_suppkey@0)] + ProjectionExec: expr=[s_suppkey@0 as s_suppkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: s_comment@1 LIKE %Customer%Complaints% + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_comment], predicate=s_comment@6 LIKE %Customer%Complaints% + 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=CollectLeft, 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 + 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={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/part.parquet]]}, projection=[p_partkey, p_brand, p_type, p_size], predicate=p_brand@3 != Brand#14 AND p_type@4 NOT LIKE SMALL PLATED% AND Use p_size@5 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) }]), 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 (14, 6, 31, 49, 15, 47, 41, 5)] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/partsupp.parquet:0..41091907]]}, projection=[ps_partkey, ps_suppkey] RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_partkey, ps_suppkey] - -Query Stage #1 (4 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) - 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: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_brand@3 != Brand#14 AND p_type@4 NOT LIKE SMALL PLATED% AND Use p_size@5 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) }]), pruning_predicate=(p_brand_min@0 != Brand#14 OR Brand#14 != p_brand_max@1) AND (p_size_min@2 <= 14 AND 14 <= p_size_max@3 OR p_size_min@2 <= 6 AND 6 <= p_size_max@3 OR p_size_min@2 <= 5 AND 5 <= p_size_max@3 OR p_size_min@2 <= 31 AND 31 <= p_size_max@3 OR p_size_min@2 <= 49 AND 49 <= p_size_max@3 OR p_size_min@2 <= 15 AND 15 <= p_size_max@3 OR p_size_min@2 <= 41 AND 41 <= p_size_max@3 OR p_size_min@2 <= 47 AND 47 <= p_size_max@3), projection=[p_partkey, p_brand, p_type, p_size] - -Query Stage #2 (4 -> 4): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 4)) - ProjectionExec: expr=[ps_suppkey@1 as ps_suppkey, p_brand@3 as p_brand, p_type@4 as p_type, p_size@5 as p_size] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "ps_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) - -Query Stage #3 (4 -> 4): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - ProjectionExec: expr=[s_suppkey@0 as s_suppkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: s_comment@1 LIKE %Customer%Complaints% - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, predicate=s_comment@6 LIKE %Customer%Complaints%, projection=[s_suppkey, s_comment] - -Query Stage #4 (4 -> 4): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "group_alias_0", index: 0 }, Column { name: "group_alias_1", index: 1 }, Column { name: "group_alias_2", index: 2 }, Column { name: "alias1", index: 3 }], 4)) - AggregateExec: mode=Partial, gby=[p_brand@1 as group_alias_0, p_type@2 as group_alias_1, p_size@3 as group_alias_2, ps_suppkey@0 as alias1], aggr=[] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(Column { name: "ps_suppkey", index: 0 }, Column { name: "s_suppkey", index: 0 })] +Query Stage #0 (1 -> 1): +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=[false] + 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=Single, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] + AggregateExec: mode=Single, 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 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - -Query Stage #5 (4 -> 4): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "group_alias_0", index: 0 }, Column { name: "group_alias_1", index: 1 }, Column { name: "group_alias_2", index: 2 }], 4)) - AggregateExec: mode=Partial, gby=[group_alias_0@0 as group_alias_0, group_alias_1@1 as group_alias_1, group_alias_2@2 as group_alias_2], aggr=[COUNT(alias1)] - AggregateExec: mode=FinalPartitioned, gby=[group_alias_0@0 as group_alias_0, group_alias_1@1 as group_alias_1, group_alias_2@2 as group_alias_2, alias1@3 as alias1], aggr=[] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "group_alias_0", index: 0 }, Column { name: "group_alias_1", index: 1 }, Column { name: "group_alias_2", index: 2 }, Column { name: "alias1", index: 3 }], 4)) - -Query Stage #6 (4 -> 4): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "part.p_brand", index: 0 }, Column { name: "part.p_type", index: 1 }, Column { name: "part.p_size", index: 2 }], 4)) - SortExec: expr=[supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST] - ProjectionExec: expr=[group_alias_0@0 as part.p_brand, group_alias_1@1 as part.p_type, group_alias_2@2 as part.p_size, COUNT(alias1)@3 as supplier_cnt] - AggregateExec: mode=FinalPartitioned, gby=[group_alias_0@0 as group_alias_0, group_alias_1@1 as group_alias_1, group_alias_2@2 as group_alias_2], aggr=[COUNT(alias1)] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "group_alias_0", index: 0 }, Column { name: "group_alias_1", index: 1 }, Column { name: "group_alias_2", index: 2 }], 4)) - -Query Stage #7 (4 -> 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: "part.p_brand", index: 0 }, Column { name: "part.p_type", index: 1 }, Column { name: "part.p_size", index: 2 }], 4)) + HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(s_suppkey@0, ps_suppkey@0)] + ProjectionExec: expr=[s_suppkey@0 as s_suppkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: s_comment@1 LIKE %Customer%Complaints% + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_comment], predicate=s_comment@6 LIKE %Customer%Complaints% + 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=CollectLeft, 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 + 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={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/part.parquet]]}, projection=[p_partkey, p_brand, p_type, p_size], predicate=p_brand@3 != Brand#14 AND p_type@4 NOT LIKE SMALL PLATED% AND Use p_size@5 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) }]), 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 (14, 6, 31, 49, 15, 47, 41, 5)] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/partsupp.parquet:0..41091907]]}, projection=[ps_partkey, ps_suppkey] diff --git a/testdata/expected-plans/q17.txt b/testdata/expected-plans/q17.txt index 3604601..92f4079 100644 --- a/testdata/expected-plans/q17.txt +++ b/testdata/expected-plans/q17.txt @@ -1,100 +1,57 @@ DataFusion Logical Plan ======================= -Projection: CAST(SUM(lineitem.l_extendedprice) AS Float64) / Float64(7) AS avg_yearly - Aggregate: groupBy=[[]], aggr=[[SUM(lineitem.l_extendedprice)]] +Projection: CAST(sum(lineitem.l_extendedprice) AS Float64) / Float64(7) AS avg_yearly + Aggregate: groupBy=[[]], aggr=[[sum(lineitem.l_extendedprice)]] Projection: lineitem.l_extendedprice - Filter: CAST(lineitem.l_quantity AS Decimal128(30, 15)) < CAST(__scalar_sq_1.__value AS Decimal128(30, 15)) AND __scalar_sq_1.l_partkey = lineitem.l_partkey - Projection: lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, __scalar_sq_1.l_partkey, __scalar_sq_1.__value - Inner Join: part.p_partkey = __scalar_sq_1.l_partkey - Filter: part.p_partkey = lineitem.l_partkey AND lineitem.l_partkey = part.p_partkey - Inner Join: lineitem.l_partkey = part.p_partkey - TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice] - Projection: part.p_partkey - Filter: part.p_brand = Utf8("Brand#42") AND part.p_container = Utf8("LG BAG") - TableScan: part projection=[p_partkey, p_brand, p_container], partial_filters=[part.p_brand = Utf8("Brand#42"), part.p_container = Utf8("LG BAG")] - SubqueryAlias: __scalar_sq_1 - Projection: lineitem.l_partkey, Float64(0.2) * CAST(AVG(lineitem.l_quantity) AS Float64) AS __value - Aggregate: groupBy=[[lineitem.l_partkey]], aggr=[[AVG(lineitem.l_quantity)]] - TableScan: lineitem projection=[l_partkey, l_quantity] + Inner Join: part.p_partkey = __scalar_sq_1.l_partkey Filter: CAST(lineitem.l_quantity AS Decimal128(30, 15)) < __scalar_sq_1.Float64(0.2) * avg(lineitem.l_quantity) + Projection: lineitem.l_quantity, lineitem.l_extendedprice, part.p_partkey + Inner Join: lineitem.l_partkey = part.p_partkey + TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice] + Projection: part.p_partkey + Filter: part.p_brand = Utf8("Brand#42") AND part.p_container = Utf8("LG BAG") + TableScan: part projection=[p_partkey, p_brand, p_container], partial_filters=[part.p_brand = Utf8("Brand#42"), part.p_container = Utf8("LG BAG")] + SubqueryAlias: __scalar_sq_1 + Projection: CAST(Float64(0.2) * CAST(avg(lineitem.l_quantity) AS Float64) AS Decimal128(30, 15)), lineitem.l_partkey + Aggregate: groupBy=[[lineitem.l_partkey]], aggr=[[avg(lineitem.l_quantity)]] + TableScan: lineitem projection=[l_partkey, l_quantity] DataFusion Physical Plan ======================== -ProjectionExec: expr=[CAST(SUM(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly] - AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice)] - CoalescePartitionsExec - AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice)] - ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice] +ProjectionExec: expr=[CAST(sum(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly] + AggregateExec: mode=Single, gby=[], aggr=[sum(lineitem.l_extendedprice)] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * avg(lineitem.l_quantity)@1, projection=[l_extendedprice@1] + ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@0 as p_partkey] CoalesceBatchesExec: target_batch_size=8192 - FilterExec: CAST(l_quantity@1 AS Decimal128(30, 15)) < CAST(__value@4 AS Decimal128(30, 15)) AND l_partkey@3 = l_partkey@0 - ProjectionExec: expr=[l_partkey@0 as l_partkey, l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_partkey@4 as l_partkey, __value@5 as __value] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_partkey@0, l_quantity@2, l_extendedprice@3] + ProjectionExec: expr=[p_partkey@0 as p_partkey] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 3 }, Column { name: "l_partkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: p_partkey@3 = l_partkey@0 AND l_partkey@0 = p_partkey@3 - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_partkey, l_quantity, l_extendedprice] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[p_partkey@0 as p_partkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: p_brand@1 = Brand#42 AND p_container@2 = LG BAG - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_brand@3 = Brand#42 AND p_container@6 = LG BAG, pruning_predicate=p_brand_min@0 <= Brand#42 AND Brand#42 <= p_brand_max@1 AND p_container_min@2 <= LG BAG AND LG BAG <= p_container_max@3, projection=[p_partkey, p_brand, p_container] - ProjectionExec: expr=[l_partkey@0 as l_partkey, 0.2 * CAST(AVG(lineitem.l_quantity)@1 AS Float64) as __value] - AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_partkey, l_quantity] + FilterExec: p_brand@1 = Brand#42 AND p_container@2 = LG BAG + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/part.parquet]]}, projection=[p_partkey, p_brand, p_container], predicate=p_brand@3 = Brand#42 AND p_container@6 = LG BAG, pruning_predicate=CASE WHEN p_brand_null_count@2 = p_brand_row_count@3 THEN false ELSE p_brand_min@0 <= Brand#42 AND Brand#42 <= p_brand_max@1 END AND CASE WHEN p_container_null_count@6 = p_container_row_count@7 THEN false ELSE p_container_min@4 <= LG BAG AND LG BAG <= p_container_max@5 END, required_guarantees=[p_brand in (Brand#42), p_container in (LG BAG)] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_partkey, l_quantity, l_extendedprice] + ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey] + AggregateExec: mode=Single, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_partkey, l_quantity] RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_partkey, l_quantity, l_extendedprice] - -Query Stage #1 (4 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) - ProjectionExec: expr=[p_partkey@0 as p_partkey] +Query Stage #0 (1 -> 1): +ProjectionExec: expr=[CAST(sum(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly] + AggregateExec: mode=Single, gby=[], aggr=[sum(lineitem.l_extendedprice)] CoalesceBatchesExec: target_batch_size=8192 - FilterExec: p_brand@1 = Brand#42 AND p_container@2 = LG BAG - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_brand@3 = Brand#42 AND p_container@6 = LG BAG, pruning_predicate=p_brand_min@0 <= Brand#42 AND Brand#42 <= p_brand_max@1 AND p_container_min@2 <= LG BAG AND LG BAG <= p_container_max@3, projection=[p_partkey, p_brand, p_container] - -Query Stage #2 (4 -> 4): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4)) - AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_partkey, l_quantity] - -Query Stage #3 (4 -> 4): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([], 4)) - AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice)] - ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: CAST(l_quantity@1 AS Decimal128(30, 15)) < CAST(__value@4 AS Decimal128(30, 15)) AND l_partkey@3 = l_partkey@0 - ProjectionExec: expr=[l_partkey@0 as l_partkey, l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_partkey@4 as l_partkey, __value@5 as __value] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 3 }, Column { name: "l_partkey", index: 0 })] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * avg(lineitem.l_quantity)@1, projection=[l_extendedprice@1] + ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@0 as p_partkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_partkey@0, l_quantity@2, l_extendedprice@3] + ProjectionExec: expr=[p_partkey@0 as p_partkey] CoalesceBatchesExec: target_batch_size=8192 - FilterExec: p_partkey@3 = l_partkey@0 AND l_partkey@0 = p_partkey@3 - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) - ProjectionExec: expr=[l_partkey@0 as l_partkey, 0.2 * CAST(AVG(lineitem.l_quantity)@1 AS Float64) as __value] - AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4)) - -Query Stage #4 (1 -> 1): -ProjectionExec: expr=[CAST(SUM(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly] - AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice)] - CoalescePartitionsExec - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([], 4)) + FilterExec: p_brand@1 = Brand#42 AND p_container@2 = LG BAG + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/part.parquet]]}, projection=[p_partkey, p_brand, p_container], predicate=p_brand@3 = Brand#42 AND p_container@6 = LG BAG, pruning_predicate=CASE WHEN p_brand_null_count@2 = p_brand_row_count@3 THEN false ELSE p_brand_min@0 <= Brand#42 AND Brand#42 <= p_brand_max@1 END AND CASE WHEN p_container_null_count@6 = p_container_row_count@7 THEN false ELSE p_container_min@4 <= LG BAG AND LG BAG <= p_container_max@5 END, required_guarantees=[p_brand in (Brand#42), p_container in (LG BAG)] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_partkey, l_quantity, l_extendedprice] + ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey] + AggregateExec: mode=Single, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_partkey, l_quantity] diff --git a/testdata/expected-plans/q18.txt b/testdata/expected-plans/q18.txt index f2ae3d8..57a45fa 100644 --- a/testdata/expected-plans/q18.txt +++ b/testdata/expected-plans/q18.txt @@ -3,7 +3,7 @@ DataFusion Logical Plan Limit: skip=0, fetch=100 Sort: orders.o_totalprice DESC NULLS FIRST, orders.o_orderdate ASC NULLS LAST, fetch=100 - Aggregate: groupBy=[[customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice]], aggr=[[SUM(lineitem.l_quantity)]] + Aggregate: groupBy=[[customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice]], aggr=[[sum(lineitem.l_quantity)]] LeftSemi Join: orders.o_orderkey = __correlated_sq_1.l_orderkey Projection: customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity Inner Join: orders.o_orderkey = lineitem.l_orderkey @@ -13,107 +13,49 @@ Limit: skip=0, fetch=100 TableScan: orders projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate] TableScan: lineitem projection=[l_orderkey, l_quantity] SubqueryAlias: __correlated_sq_1 - Projection: lineitem.l_orderkey AS l_orderkey - Filter: SUM(lineitem.l_quantity) > Decimal128(Some(31300),21,2) - Aggregate: groupBy=[[lineitem.l_orderkey]], aggr=[[SUM(lineitem.l_quantity)]] + Projection: lineitem.l_orderkey + Filter: sum(lineitem.l_quantity) > Decimal128(Some(31300),21,2) + Aggregate: groupBy=[[lineitem.l_orderkey]], aggr=[[sum(lineitem.l_quantity)]] TableScan: lineitem projection=[l_orderkey, l_quantity] DataFusion Physical Plan ======================== -GlobalLimitExec: skip=0, fetch=100 - SortPreservingMergeExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] - SortExec: fetch=100, expr=[o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] - AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[SUM(lineitem.l_quantity)] +SortExec: TopK(fetch=100), expr=[o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[false] + AggregateExec: mode=Single, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[sum(lineitem.l_quantity)] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@2)] + ProjectionExec: expr=[l_orderkey@0 as l_orderkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: sum(lineitem.l_quantity)@1 > Some(31300),21,2 + AggregateExec: mode=Single, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_quantity] CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: 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 }], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[SUM(lineitem.l_quantity)] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "o_orderkey", index: 2 }, Column { name: "l_orderkey", index: 0 })] - ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, o_orderkey@2 as o_orderkey, o_totalprice@3 as o_totalprice, o_orderdate@4 as o_orderdate, l_quantity@6 as l_quantity] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 2 }, Column { name: "l_orderkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 2 }], 4), input_partitions=4 - ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, o_orderkey@2 as o_orderkey, o_totalprice@4 as o_totalprice, o_orderdate@5 as o_orderdate] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, projection=[c_custkey, c_name] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_orderkey, l_quantity] - ProjectionExec: expr=[l_orderkey@0 as l_orderkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: SUM(lineitem.l_quantity)@1 > Some(31300),21,2 - AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_orderkey, l_quantity] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/customer.parquet:0..13211178]]}, projection=[c_custkey, c_name] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_quantity] RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, projection=[c_custkey, c_name] - -Query Stage #1 (4 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate] - -Query Stage #2 (4 -> 4): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_orderkey", index: 2 }], 4)) - ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, o_orderkey@2 as o_orderkey, o_totalprice@4 as o_totalprice, o_orderdate@5 as o_orderdate] +Query Stage #0 (1 -> 1): +SortExec: TopK(fetch=100), expr=[o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[false] + AggregateExec: mode=Single, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[sum(lineitem.l_quantity)] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) - -Query Stage #3 (4 -> 4): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_orderkey, l_quantity] - -Query Stage #4 (4 -> 4): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_orderkey, l_quantity] - -Query Stage #5 (4 -> 4): -ShuffleWriterExec(stage_id=5, output_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 }], 4)) - AggregateExec: mode=Partial, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[SUM(lineitem.l_quantity)] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "o_orderkey", index: 2 }, Column { name: "l_orderkey", index: 0 })] - ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, o_orderkey@2 as o_orderkey, o_totalprice@3 as o_totalprice, o_orderdate@4 as o_orderdate, l_quantity@6 as l_quantity] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 2 }, Column { name: "l_orderkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_orderkey", index: 2 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) + HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@2)] ProjectionExec: expr=[l_orderkey@0 as l_orderkey] CoalesceBatchesExec: target_batch_size=8192 - FilterExec: SUM(lineitem.l_quantity)@1 > Some(31300),21,2 - AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - -Query Stage #6 (4 -> 4): -ShuffleWriterExec(stage_id=6, output_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 }], 4)) - SortExec: fetch=100, expr=[o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] - AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[SUM(lineitem.l_quantity)] - 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 }], 4)) - -Query Stage #7 (1 -> 1): -GlobalLimitExec: skip=0, fetch=100 - SortPreservingMergeExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] - 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 }], 4)) + FilterExec: sum(lineitem.l_quantity)@1 > Some(31300),21,2 + AggregateExec: mode=Single, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_quantity] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/customer.parquet:0..13211178]]}, projection=[c_custkey, c_name] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_quantity] diff --git a/testdata/expected-plans/q19.txt b/testdata/expected-plans/q19.txt index e61791f..66c9ced 100644 --- a/testdata/expected-plans/q19.txt +++ b/testdata/expected-plans/q19.txt @@ -1,76 +1,44 @@ DataFusion Logical Plan ======================= -Projection: SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue - Aggregate: groupBy=[[]], aggr=[[SUM(CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] +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 - 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_quantity, lineitem.l_extendedprice, lineitem.l_discount, part.p_brand, part.p_size, part.p_container - Inner Join: lineitem.l_partkey = part.p_partkey - Projection: lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount - Filter: (lineitem.l_quantity >= Decimal128(Some(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 REG") OR lineitem.l_shipmode = Utf8("AIR")) AND lineitem.l_shipinstruct = Utf8("DELIVER IN PERSON") - TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], partial_filters=[lineitem.l_shipmode = Utf8("AIR REG") OR lineitem.l_shipmode = Utf8("AIR"), 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)] + 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)] - ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue] + AggregateExec: mode=Single, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, 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 + 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={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/part.parquet]]}, projection=[p_partkey, p_brand, p_size, p_container], predicate=p_size@5 >= 1 AND (p_brand@3 = Brand#21 AND Use p_container@6 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@5 <= 5 OR p_brand@3 = Brand#13 AND Use p_container@6 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@5 <= 10 OR p_brand@3 = Brand#52 AND Use p_container@6 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@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=[] + ProjectionExec: expr=[l_partkey@0 as l_partkey, l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount] CoalesceBatchesExec: target_batch_size=8192 - FilterExec: p_brand@3 = Brand#21 AND Use p_container@5 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@4 <= 5 OR p_brand@3 = Brand#13 AND Use p_container@5 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@4 <= 10 OR p_brand@3 = Brand#52 AND Use p_container@5 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@4 <= 15 - ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, p_brand@5 as p_brand, p_size@6 as p_size, p_container@7 as p_container] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[l_partkey@0 as l_partkey, l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount] - 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 REG OR l_shipmode@5 = AIR) AND l_shipinstruct@4 = DELIVER IN PERSON - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=(l_shipmode@14 = AIR REG OR l_shipmode@14 = AIR) 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=(l_shipmode_min@0 <= AIR REG AND AIR REG <= l_shipmode_max@1 OR l_shipmode_min@0 <= AIR AND AIR <= l_shipmode_max@1) AND l_shipinstruct_min@2 <= DELIVER IN PERSON AND DELIVER IN PERSON <= l_shipinstruct_max@3 AND (l_quantity_max@4 >= Some(800),11,2 AND l_quantity_min@5 <= Some(1800),11,2 OR l_quantity_max@4 >= Some(2000),11,2 AND l_quantity_min@5 <= Some(3000),11,2 OR l_quantity_max@4 >= Some(3000),11,2 AND l_quantity_min@5 <= Some(4000),11,2), projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4), input_partitions=4 - 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: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_size@5 >= 1 AND (p_brand@3 = Brand#21 AND Use p_container@6 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@5 <= 5 OR p_brand@3 = Brand#13 AND Use p_container@6 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@5 <= 10 OR p_brand@3 = Brand#52 AND Use p_container@6 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@5 <= 15), pruning_predicate=p_size_max@0 >= 1 AND (p_brand_min@1 <= Brand#21 AND Brand#21 <= p_brand_max@2 AND (p_container_min@3 <= SM CASE AND SM CASE <= p_container_max@4 OR p_container_min@3 <= SM BOX AND SM BOX <= p_container_max@4 OR p_container_min@3 <= SM PACK AND SM PACK <= p_container_max@4 OR p_container_min@3 <= SM PKG AND SM PKG <= p_container_max@4) AND p_size_min@5 <= 5 OR p_brand_min@1 <= Brand#13 AND Brand#13 <= p_brand_max@2 AND (p_container_min@3 <= MED BAG AND MED BAG <= p_container_max@4 OR p_container_min@3 <= MED BOX AND MED BOX <= p_container_max@4 OR p_container_min@3 <= MED PKG AND MED PKG <= p_container_max@4 OR p_container_min@3 <= MED PACK AND MED PACK <= p_container_max@4) AND p_size_min@5 <= 10 OR p_brand_min@1 <= Brand#52 AND Brand#52 <= p_brand_max@2 AND (p_container_min@3 <= LG CASE AND LG CASE <= p_container_max@4 OR p_container_min@3 <= LG BOX AND LG BOX <= p_container_max@4 OR p_container_min@3 <= LG PACK AND LG PACK <= p_container_max@4 OR p_container_min@3 <= LG PKG AND LG PKG <= p_container_max@4) AND p_size_min@5 <= 15), projection=[p_partkey, p_brand, p_size, p_container] + 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 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, 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, AIR REG), l_shipinstruct in (DELIVER IN PERSON)] RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4)) - ProjectionExec: expr=[l_partkey@0 as l_partkey, l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount] +Query Stage #0 (1 -> 1): +ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue] + AggregateExec: mode=Single, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 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 REG OR l_shipmode@5 = AIR) AND l_shipinstruct@4 = DELIVER IN PERSON - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=(l_shipmode@14 = AIR REG OR l_shipmode@14 = AIR) 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=(l_shipmode_min@0 <= AIR REG AND AIR REG <= l_shipmode_max@1 OR l_shipmode_min@0 <= AIR AND AIR <= l_shipmode_max@1) AND l_shipinstruct_min@2 <= DELIVER IN PERSON AND DELIVER IN PERSON <= l_shipinstruct_max@3 AND (l_quantity_max@4 >= Some(800),11,2 AND l_quantity_min@5 <= Some(1800),11,2 OR l_quantity_max@4 >= Some(2000),11,2 AND l_quantity_min@5 <= Some(3000),11,2 OR l_quantity_max@4 >= Some(3000),11,2 AND l_quantity_min@5 <= Some(4000),11,2), projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode] - -Query Stage #1 (4 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) - 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: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_size@5 >= 1 AND (p_brand@3 = Brand#21 AND Use p_container@6 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@5 <= 5 OR p_brand@3 = Brand#13 AND Use p_container@6 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@5 <= 10 OR p_brand@3 = Brand#52 AND Use p_container@6 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@5 <= 15), pruning_predicate=p_size_max@0 >= 1 AND (p_brand_min@1 <= Brand#21 AND Brand#21 <= p_brand_max@2 AND (p_container_min@3 <= SM CASE AND SM CASE <= p_container_max@4 OR p_container_min@3 <= SM BOX AND SM BOX <= p_container_max@4 OR p_container_min@3 <= SM PACK AND SM PACK <= p_container_max@4 OR p_container_min@3 <= SM PKG AND SM PKG <= p_container_max@4) AND p_size_min@5 <= 5 OR p_brand_min@1 <= Brand#13 AND Brand#13 <= p_brand_max@2 AND (p_container_min@3 <= MED BAG AND MED BAG <= p_container_max@4 OR p_container_min@3 <= MED BOX AND MED BOX <= p_container_max@4 OR p_container_min@3 <= MED PKG AND MED PKG <= p_container_max@4 OR p_container_min@3 <= MED PACK AND MED PACK <= p_container_max@4) AND p_size_min@5 <= 10 OR p_brand_min@1 <= Brand#52 AND Brand#52 <= p_brand_max@2 AND (p_container_min@3 <= LG CASE AND LG CASE <= p_container_max@4 OR p_container_min@3 <= LG BOX AND LG BOX <= p_container_max@4 OR p_container_min@3 <= LG PACK AND LG PACK <= p_container_max@4 OR p_container_min@3 <= LG PKG AND LG PKG <= p_container_max@4) AND p_size_min@5 <= 15), projection=[p_partkey, p_brand, p_size, p_container] - -Query Stage #2 (4 -> 4): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([], 4)) - AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: p_brand@3 = Brand#21 AND Use p_container@5 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@4 <= 5 OR p_brand@3 = Brand#13 AND Use p_container@5 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@4 <= 10 OR p_brand@3 = Brand#52 AND Use p_container@5 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@4 <= 15 - ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, p_brand@5 as p_brand, p_size@6 as p_size, p_container@7 as p_container] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) - -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([], 4)) + HashJoinExec: mode=CollectLeft, 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 + 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={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/part.parquet]]}, projection=[p_partkey, p_brand, p_size, p_container], predicate=p_size@5 >= 1 AND (p_brand@3 = Brand#21 AND Use p_container@6 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@5 <= 5 OR p_brand@3 = Brand#13 AND Use p_container@6 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@5 <= 10 OR p_brand@3 = Brand#52 AND Use p_container@6 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@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=[] + ProjectionExec: expr=[l_partkey@0 as l_partkey, l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount] + 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 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, 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, AIR REG), l_shipinstruct in (DELIVER IN PERSON)] diff --git a/testdata/expected-plans/q2.txt b/testdata/expected-plans/q2.txt index 64df0f1..bc2c04a 100644 --- a/testdata/expected-plans/q2.txt +++ b/testdata/expected-plans/q2.txt @@ -4,12 +4,12 @@ DataFusion Logical Plan Limit: skip=0, fetch=100 Sort: supplier.s_acctbal DESC NULLS FIRST, nation.n_name ASC NULLS LAST, supplier.s_name ASC NULLS LAST, part.p_partkey ASC NULLS LAST, fetch=100 Projection: supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment - Inner Join: part.p_partkey = __scalar_sq_1.ps_partkey, partsupp.ps_supplycost = __scalar_sq_1.__value - Projection: part.p_partkey, part.p_mfgr, partsupp.ps_supplycost, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name + Inner Join: part.p_partkey = __scalar_sq_1.ps_partkey, partsupp.ps_supplycost = __scalar_sq_1.min(partsupp.ps_supplycost) + Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost, nation.n_name Inner Join: nation.n_regionkey = region.r_regionkey - Projection: part.p_partkey, part.p_mfgr, partsupp.ps_supplycost, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey + Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost, nation.n_name, nation.n_regionkey Inner Join: supplier.s_nationkey = nation.n_nationkey - Projection: part.p_partkey, part.p_mfgr, partsupp.ps_supplycost, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment + Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost Inner Join: partsupp.ps_suppkey = supplier.s_suppkey Projection: part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, partsupp.ps_supplycost Inner Join: part.p_partkey = partsupp.ps_partkey @@ -23,8 +23,8 @@ Limit: skip=0, fetch=100 Filter: region.r_name = Utf8("ASIA") TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("ASIA")] SubqueryAlias: __scalar_sq_1 - Projection: partsupp.ps_partkey, MIN(partsupp.ps_supplycost) AS __value - Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[MIN(partsupp.ps_supplycost)]] + Projection: min(partsupp.ps_supplycost), partsupp.ps_partkey + Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[min(partsupp.ps_supplycost)]] Projection: partsupp.ps_partkey, partsupp.ps_supplycost Inner Join: nation.n_regionkey = region.r_regionkey Projection: partsupp.ps_partkey, partsupp.ps_supplycost, nation.n_regionkey @@ -41,234 +41,93 @@ Limit: skip=0, fetch=100 DataFusion Physical Plan ======================== -GlobalLimitExec: skip=0, fetch=100 - SortPreservingMergeExec: [s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST] - SortExec: fetch=100, expr=[s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST] - ProjectionExec: expr=[s_acctbal@6 as s_acctbal, s_name@3 as s_name, n_name@8 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@4 as s_address, s_phone@5 as s_phone, s_comment@7 as s_comment] +SortExec: TopK(fetch=100), expr=[s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST], preserve_partitioning=[false] + ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@7 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, min(partsupp.ps_supplycost)@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 0 }, Column { name: "ps_partkey", index: 0 }), (Column { name: "ps_supplycost", index: 2 }, Column { name: "__value", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 0 }, Column { name: "ps_supplycost", index: 2 }], 4), input_partitions=4 - ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, ps_supplycost@2 as ps_supplycost, s_name@3 as s_name, s_address@4 as s_address, s_phone@5 as s_phone, s_acctbal@6 as s_acctbal, s_comment@7 as s_comment, n_name@8 as n_name] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "n_regionkey", index: 9 }, Column { name: "r_regionkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "n_regionkey", index: 9 }], 4), input_partitions=4 - ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, ps_supplycost@2 as ps_supplycost, s_name@3 as s_name, s_address@4 as s_address, s_phone@6 as s_phone, s_acctbal@7 as s_acctbal, s_comment@8 as s_comment, n_name@10 as n_name, n_regionkey@11 as n_regionkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 5 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 5 }], 4), input_partitions=4 - ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, ps_supplycost@3 as ps_supplycost, s_name@5 as s_name, s_address@6 as s_address, s_nationkey@7 as s_nationkey, s_phone@8 as s_phone, s_acctbal@9 as s_acctbal, s_comment@10 as s_comment] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "ps_suppkey", index: 2 }, Column { name: "s_suppkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "ps_suppkey", index: 2 }], 4), input_partitions=4 - ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, ps_suppkey@3 as ps_suppkey, ps_supplycost@4 as ps_supplycost] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 0 }, Column { name: "ps_partkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: p_size@3 = 48 AND p_type@2 LIKE %TIN - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_size@5 = 48 AND p_type@4 LIKE %TIN, pruning_predicate=p_size_min@0 <= 48 AND 48 <= p_size_max@1, projection=[p_partkey, p_mfgr, p_type, p_size] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=1 - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_name, n_regionkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4), input_partitions=4 - RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - ProjectionExec: expr=[r_regionkey@0 as r_regionkey] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@9)], projection=[p_partkey@1, p_mfgr@2, s_name@3, s_address@4, s_phone@5, s_acctbal@6, s_comment@7, ps_supplycost@8, n_name@9] + ProjectionExec: expr=[r_regionkey@0 as r_regionkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: r_name@1 = ASIA + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/region.parquet]]}, 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)] + ProjectionExec: expr=[p_partkey@2 as p_partkey, p_mfgr@3 as p_mfgr, s_name@4 as s_name, s_address@5 as s_address, s_phone@6 as s_phone, s_acctbal@7 as s_acctbal, s_comment@8 as s_comment, ps_supplycost@9 as ps_supplycost, n_name@0 as n_name, n_regionkey@1 as n_regionkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@4)], projection=[n_name@1, n_regionkey@2, p_partkey@3, p_mfgr@4, s_name@5, s_address@6, s_phone@8, s_acctbal@9, s_comment@10, ps_supplycost@11] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, projection=[n_nationkey, n_name, n_regionkey] + ProjectionExec: expr=[p_partkey@6 as p_partkey, p_mfgr@7 as p_mfgr, s_name@0 as s_name, s_address@1 as s_address, s_nationkey@2 as s_nationkey, s_phone@3 as s_phone, s_acctbal@4 as s_acctbal, s_comment@5 as s_comment, ps_supplycost@8 as ps_supplycost] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@2)], projection=[s_name@1, s_address@2, s_nationkey@3, s_phone@4, s_acctbal@5, s_comment@6, p_partkey@7, p_mfgr@8, ps_supplycost@10] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_suppkey@3, ps_supplycost@4] + ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] CoalesceBatchesExec: target_batch_size=8192 - FilterExec: r_name@1 = ASIA - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/region.parquet/part-0.parquet]]}, predicate=r_name@1 = ASIA, pruning_predicate=r_name_min@0 <= ASIA AND ASIA <= r_name_max@1, projection=[r_regionkey, r_name] + FilterExec: p_size@3 = 48 AND p_type@2 LIKE %TIN + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/part.parquet]]}, projection=[p_partkey, p_mfgr, p_type, p_size], predicate=p_size@5 = 48 AND p_type@4 LIKE %TIN, pruning_predicate=CASE WHEN p_size_null_count@2 = p_size_row_count@3 THEN false ELSE p_size_min@0 <= 48 AND 48 <= p_size_max@1 END, required_guarantees=[p_size in (48)] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/partsupp.parquet:0..41091907]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost] + ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] + AggregateExec: mode=Single, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "ps_partkey", index: 0 }, Column { name: "__value", index: 1 }], 4), input_partitions=4 - ProjectionExec: expr=[ps_partkey@0 as ps_partkey, MIN(partsupp.ps_supplycost)@1 as __value] - AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] - ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@1 as ps_supplycost] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "n_regionkey", index: 2 }, Column { name: "r_regionkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "n_regionkey", index: 2 }], 4), input_partitions=4 - ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@1 as ps_supplycost, n_regionkey@4 as n_regionkey] - ProjectionExec: expr=[ps_partkey@2 as ps_partkey, ps_supplycost@3 as ps_supplycost, s_nationkey@4 as s_nationkey, n_nationkey@0 as n_nationkey, n_regionkey@1 as n_regionkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "n_nationkey", index: 0 }, Column { name: "s_nationkey", index: 2 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=1 - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_regionkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 4), input_partitions=4 - ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@2 as ps_supplycost, s_nationkey@4 as s_nationkey] - ProjectionExec: expr=[ps_partkey@2 as ps_partkey, ps_suppkey@3 as ps_suppkey, ps_supplycost@4 as ps_supplycost, s_suppkey@0 as s_suppkey, s_nationkey@1 as s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "ps_suppkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4), input_partitions=4 - RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - ProjectionExec: expr=[r_regionkey@0 as r_regionkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: r_name@1 = ASIA - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/region.parquet/part-0.parquet]]}, predicate=r_name@1 = ASIA, pruning_predicate=r_name_min@0 <= ASIA AND ASIA <= r_name_max@1, projection=[r_regionkey, r_name] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@2)], projection=[ps_partkey@1, ps_supplycost@2] + ProjectionExec: expr=[r_regionkey@0 as r_regionkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: r_name@1 = ASIA + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/region.parquet]]}, 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)] + ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, n_regionkey@0 as n_regionkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_regionkey@1, ps_partkey@2, ps_supplycost@3] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, projection=[n_nationkey, n_regionkey] + ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@1)], projection=[s_nationkey@1, ps_partkey@2, ps_supplycost@4] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_nationkey] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/partsupp.parquet:0..41091907]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost] RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) - ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: p_size@3 = 48 AND p_type@2 LIKE %TIN - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_size@5 = 48 AND p_type@4 LIKE %TIN, pruning_predicate=p_size_min@0 <= 48 AND 48 <= p_size_max@1, projection=[p_partkey, p_mfgr, p_type, p_size] - -Query Stage #1 (4 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost] - -Query Stage #2 (4 -> 4): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "ps_suppkey", index: 2 }], 4)) - ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, ps_suppkey@3 as ps_suppkey, ps_supplycost@4 as ps_supplycost] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 0 }, Column { name: "ps_partkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) - -Query Stage #3 (4 -> 4): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] - -Query Stage #4 (4 -> 4): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "s_nationkey", index: 5 }], 4)) - ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, ps_supplycost@3 as ps_supplycost, s_name@5 as s_name, s_address@6 as s_address, s_nationkey@7 as s_nationkey, s_phone@8 as s_phone, s_acctbal@9 as s_acctbal, s_comment@10 as s_comment] +Query Stage #0 (1 -> 1): +SortExec: TopK(fetch=100), expr=[s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST], preserve_partitioning=[false] + ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@7 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "ps_suppkey", index: 2 }, Column { name: "s_suppkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "ps_suppkey", index: 2 }], 4)) + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, min(partsupp.ps_supplycost)@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - -Query Stage #5 (1 -> 4): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_name, n_regionkey] - -Query Stage #6 (4 -> 4): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "n_regionkey", index: 9 }], 4)) - ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, ps_supplycost@2 as ps_supplycost, s_name@3 as s_name, s_address@4 as s_address, s_phone@6 as s_phone, s_acctbal@7 as s_acctbal, s_comment@8 as s_comment, n_name@10 as n_name, n_regionkey@11 as n_regionkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 5 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "s_nationkey", index: 5 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - -Query Stage #7 (1 -> 4): -ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4)) - ProjectionExec: expr=[r_regionkey@0 as r_regionkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: r_name@1 = ASIA - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/region.parquet/part-0.parquet]]}, predicate=r_name@1 = ASIA, pruning_predicate=r_name_min@0 <= ASIA AND ASIA <= r_name_max@1, projection=[r_regionkey, r_name] - -Query Stage #8 (4 -> 4): -ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }, Column { name: "ps_supplycost", index: 2 }], 4)) - ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, ps_supplycost@2 as ps_supplycost, s_name@3 as s_name, s_address@4 as s_address, s_phone@5 as s_phone, s_acctbal@6 as s_acctbal, s_comment@7 as s_comment, n_name@8 as n_name] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "n_regionkey", index: 9 }, Column { name: "r_regionkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "n_regionkey", index: 9 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4)) - -Query Stage #9 (1 -> 4): -ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_regionkey] - -Query Stage #10 (4 -> 4): -ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] - -Query Stage #11 (4 -> 4): -ShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost] - -Query Stage #12 (4 -> 4): -ShuffleWriterExec(stage_id=12, output_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 4)) - ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@2 as ps_supplycost, s_nationkey@4 as s_nationkey] - ProjectionExec: expr=[ps_partkey@2 as ps_partkey, ps_suppkey@3 as ps_suppkey, ps_supplycost@4 as ps_supplycost, s_suppkey@0 as s_suppkey, s_nationkey@1 as s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "ps_suppkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 4)) - -Query Stage #13 (4 -> 4): -ShuffleWriterExec(stage_id=13, output_partitioning=Hash([Column { name: "n_regionkey", index: 2 }], 4)) - ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@1 as ps_supplycost, n_regionkey@4 as n_regionkey] - ProjectionExec: expr=[ps_partkey@2 as ps_partkey, ps_supplycost@3 as ps_supplycost, s_nationkey@4 as s_nationkey, n_nationkey@0 as n_nationkey, n_regionkey@1 as n_regionkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "n_nationkey", index: 0 }, Column { name: "s_nationkey", index: 2 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=12, input_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 4)) - -Query Stage #14 (1 -> 4): -ShuffleWriterExec(stage_id=14, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4)) - ProjectionExec: expr=[r_regionkey@0 as r_regionkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: r_name@1 = ASIA - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/region.parquet/part-0.parquet]]}, predicate=r_name@1 = ASIA, pruning_predicate=r_name_min@0 <= ASIA AND ASIA <= r_name_max@1, projection=[r_regionkey, r_name] - -Query Stage #15 (4 -> 4): -ShuffleWriterExec(stage_id=15, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) - AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] - ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@1 as ps_supplycost] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "n_regionkey", index: 2 }, Column { name: "r_regionkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=13, input_partitioning=Hash([Column { name: "n_regionkey", index: 2 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=14, input_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4)) - -Query Stage #16 (4 -> 4): -ShuffleWriterExec(stage_id=16, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }, Column { name: "__value", index: 1 }], 4)) - ProjectionExec: expr=[ps_partkey@0 as ps_partkey, MIN(partsupp.ps_supplycost)@1 as __value] - AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=15, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) - -Query Stage #17 (4 -> 4): -ShuffleWriterExec(stage_id=17, output_partitioning=Hash([Column { name: "p_partkey", index: 3 }], 4)) - SortExec: fetch=100, expr=[s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST] - ProjectionExec: expr=[s_acctbal@6 as s_acctbal, s_name@3 as s_name, n_name@8 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@4 as s_address, s_phone@5 as s_phone, s_comment@7 as s_comment] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 0 }, Column { name: "ps_partkey", index: 0 }), (Column { name: "ps_supplycost", index: 2 }, Column { name: "__value", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }, Column { name: "ps_supplycost", index: 2 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=16, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }, Column { name: "__value", index: 1 }], 4)) - -Query Stage #18 (1 -> 1): -GlobalLimitExec: skip=0, fetch=100 - SortPreservingMergeExec: [s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST] - ShuffleReaderExec(stage_id=17, input_partitioning=Hash([Column { name: "p_partkey", index: 3 }], 4)) + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@9)], projection=[p_partkey@1, p_mfgr@2, s_name@3, s_address@4, s_phone@5, s_acctbal@6, s_comment@7, ps_supplycost@8, n_name@9] + ProjectionExec: expr=[r_regionkey@0 as r_regionkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: r_name@1 = ASIA + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/region.parquet]]}, 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)] + ProjectionExec: expr=[p_partkey@2 as p_partkey, p_mfgr@3 as p_mfgr, s_name@4 as s_name, s_address@5 as s_address, s_phone@6 as s_phone, s_acctbal@7 as s_acctbal, s_comment@8 as s_comment, ps_supplycost@9 as ps_supplycost, n_name@0 as n_name, n_regionkey@1 as n_regionkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@4)], projection=[n_name@1, n_regionkey@2, p_partkey@3, p_mfgr@4, s_name@5, s_address@6, s_phone@8, s_acctbal@9, s_comment@10, ps_supplycost@11] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, projection=[n_nationkey, n_name, n_regionkey] + ProjectionExec: expr=[p_partkey@6 as p_partkey, p_mfgr@7 as p_mfgr, s_name@0 as s_name, s_address@1 as s_address, s_nationkey@2 as s_nationkey, s_phone@3 as s_phone, s_acctbal@4 as s_acctbal, s_comment@5 as s_comment, ps_supplycost@8 as ps_supplycost] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@2)], projection=[s_name@1, s_address@2, s_nationkey@3, s_phone@4, s_acctbal@5, s_comment@6, p_partkey@7, p_mfgr@8, ps_supplycost@10] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_suppkey@3, ps_supplycost@4] + ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: p_size@3 = 48 AND p_type@2 LIKE %TIN + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/part.parquet]]}, projection=[p_partkey, p_mfgr, p_type, p_size], predicate=p_size@5 = 48 AND p_type@4 LIKE %TIN, pruning_predicate=CASE WHEN p_size_null_count@2 = p_size_row_count@3 THEN false ELSE p_size_min@0 <= 48 AND 48 <= p_size_max@1 END, required_guarantees=[p_size in (48)] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/partsupp.parquet:0..41091907]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost] + ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] + AggregateExec: mode=Single, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@2)], projection=[ps_partkey@1, ps_supplycost@2] + ProjectionExec: expr=[r_regionkey@0 as r_regionkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: r_name@1 = ASIA + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/region.parquet]]}, 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)] + ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, n_regionkey@0 as n_regionkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_regionkey@1, ps_partkey@2, ps_supplycost@3] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, projection=[n_nationkey, n_regionkey] + ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@1)], projection=[s_nationkey@1, ps_partkey@2, ps_supplycost@4] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_nationkey] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/partsupp.parquet:0..41091907]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost] diff --git a/testdata/expected-plans/q20.txt b/testdata/expected-plans/q20.txt index 3cdbf1c..5927d2f 100644 --- a/testdata/expected-plans/q20.txt +++ b/testdata/expected-plans/q20.txt @@ -11,156 +11,77 @@ Sort: supplier.s_name ASC NULLS LAST Filter: nation.n_name = Utf8("KENYA") TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("KENYA")] SubqueryAlias: __correlated_sq_1 - Projection: partsupp.ps_suppkey AS ps_suppkey - Filter: CAST(partsupp.ps_availqty AS Float64) > __scalar_sq_1.__value - Projection: partsupp.ps_suppkey, partsupp.ps_availqty, __scalar_sq_1.__value - Inner Join: partsupp.ps_partkey = __scalar_sq_1.l_partkey, partsupp.ps_suppkey = __scalar_sq_1.l_suppkey - LeftSemi Join: partsupp.ps_partkey = __correlated_sq_2.p_partkey - TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_availqty] - SubqueryAlias: __correlated_sq_2 - Projection: part.p_partkey AS p_partkey - Filter: part.p_name LIKE Utf8("blanched%") - TableScan: part projection=[p_partkey, p_name], partial_filters=[part.p_name LIKE Utf8("blanched%")] - SubqueryAlias: __scalar_sq_1 - Projection: lineitem.l_partkey, lineitem.l_suppkey, Float64(0.5) * CAST(SUM(lineitem.l_quantity) AS Float64) AS __value - Aggregate: groupBy=[[lineitem.l_partkey, lineitem.l_suppkey]], aggr=[[SUM(lineitem.l_quantity)]] - Projection: lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity - Filter: lineitem.l_shipdate >= Date32("8401") AND lineitem.l_shipdate < Date32("8766") - TableScan: lineitem projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("8401"), lineitem.l_shipdate < Date32("8766")] + Projection: partsupp.ps_suppkey + Inner Join: partsupp.ps_partkey = __scalar_sq_3.l_partkey, partsupp.ps_suppkey = __scalar_sq_3.l_suppkey Filter: CAST(partsupp.ps_availqty AS Float64) > __scalar_sq_3.Float64(0.5) * sum(lineitem.l_quantity) + LeftSemi Join: partsupp.ps_partkey = __correlated_sq_2.p_partkey + TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_availqty] + SubqueryAlias: __correlated_sq_2 + Projection: part.p_partkey + Filter: part.p_name LIKE Utf8("blanched%") + TableScan: part projection=[p_partkey, p_name], partial_filters=[part.p_name LIKE Utf8("blanched%")] + SubqueryAlias: __scalar_sq_3 + Projection: Float64(0.5) * CAST(sum(lineitem.l_quantity) AS Float64), lineitem.l_partkey, lineitem.l_suppkey + Aggregate: groupBy=[[lineitem.l_partkey, lineitem.l_suppkey]], aggr=[[sum(lineitem.l_quantity)]] + Projection: lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity + Filter: lineitem.l_shipdate >= Date32("1993-01-01") AND lineitem.l_shipdate < Date32("1994-01-01") + TableScan: lineitem projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("1993-01-01"), lineitem.l_shipdate < Date32("1994-01-01")] DataFusion Physical Plan ======================== -SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] - SortExec: expr=[s_name@0 ASC NULLS LAST] - ProjectionExec: expr=[s_name@1 as s_name, s_address@2 as s_address] +SortExec: expr=[s_name@0 ASC NULLS LAST], preserve_partitioning=[false] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[s_suppkey@1, s_name@2, s_address@3] + ProjectionExec: expr=[n_nationkey@0 as n_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: n_name@1 = KENYA + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, 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)] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_name, s_address, s_nationkey] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "ps_suppkey", index: 0 })] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * sum(lineitem.l_quantity)@1, projection=[ps_suppkey@1] CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address] + HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(p_partkey@0, ps_partkey@0)] + ProjectionExec: expr=[p_partkey@0 as p_partkey] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 3 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_name, s_address, s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4 - RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - ProjectionExec: expr=[n_nationkey@0 as n_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: n_name@1 = KENYA - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, predicate=n_name@1 = KENYA, pruning_predicate=n_name_min@0 <= KENYA AND KENYA <= n_name_max@1, projection=[n_nationkey, n_name] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[ps_suppkey@0 as ps_suppkey] + FilterExec: p_name@1 LIKE blanched% + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/part.parquet]]}, projection=[p_partkey, p_name], predicate=p_name@1 LIKE blanched% + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/partsupp.parquet:0..41091907]]}, projection=[ps_partkey, ps_suppkey, ps_availqty] + ProjectionExec: expr=[0.5 * CAST(sum(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * sum(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] + AggregateExec: mode=Single, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] + ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] CoalesceBatchesExec: target_batch_size=8192 - FilterExec: CAST(ps_availqty@1 AS Float64) > __value@2 - ProjectionExec: expr=[ps_suppkey@1 as ps_suppkey, ps_availqty@2 as ps_availqty, __value@5 as __value] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "ps_partkey", index: 0 }, Column { name: "l_partkey", index: 0 }), (Column { name: "ps_suppkey", index: 1 }, Column { name: "l_suppkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "ps_partkey", index: 0 }, Column { name: "ps_suppkey", index: 1 }], 4), input_partitions=4 - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "ps_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_availqty] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[p_partkey@0 as p_partkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: p_name@1 LIKE blanched% - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_name@1 LIKE blanched%, projection=[p_partkey, p_name] - ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, 0.5 * CAST(SUM(lineitem.l_quantity)@2 AS Float64) as __value] - AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_partkey", index: 0 }, Column { name: "l_suppkey", index: 1 }], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] - ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_shipdate@3 >= 8401 AND l_shipdate@3 < 8766 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate@10 >= 8401 AND l_shipdate@10 < 8766, pruning_predicate=l_shipdate_max@0 >= 8401 AND l_shipdate_min@1 < 8766, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate] + FilterExec: l_shipdate@3 >= 1993-01-01 AND l_shipdate@3 < 1994-01-01 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], predicate=l_shipdate@10 >= 1993-01-01 AND l_shipdate@10 < 1994-01-01, pruning_predicate=CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_max@0 >= 1993-01-01 END AND CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_min@3 < 1994-01-01 END, required_guarantees=[] RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_name, s_address, s_nationkey] - -Query Stage #1 (1 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - ProjectionExec: expr=[n_nationkey@0 as n_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: n_name@1 = KENYA - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, predicate=n_name@1 = KENYA, pruning_predicate=n_name_min@0 <= KENYA AND KENYA <= n_name_max@1, projection=[n_nationkey, n_name] - -Query Stage #2 (4 -> 4): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 3 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - -Query Stage #3 (4 -> 4): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_availqty] - -Query Stage #4 (4 -> 4): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) - ProjectionExec: expr=[p_partkey@0 as p_partkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: p_name@1 LIKE blanched% - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_name@1 LIKE blanched%, projection=[p_partkey, p_name] - -Query Stage #5 (4 -> 4): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }, Column { name: "ps_suppkey", index: 1 }], 4)) +Query Stage #0 (1 -> 1): +SortExec: expr=[s_name@0 ASC NULLS LAST], preserve_partitioning=[false] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "ps_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) + HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) - -Query Stage #6 (4 -> 4): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }, Column { name: "l_suppkey", index: 1 }], 4)) - AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] - ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_shipdate@3 >= 8401 AND l_shipdate@3 < 8766 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate@10 >= 8401 AND l_shipdate@10 < 8766, pruning_predicate=l_shipdate_max@0 >= 8401 AND l_shipdate_min@1 < 8766, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate] - -Query Stage #7 (4 -> 4): -ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 4)) - ProjectionExec: expr=[ps_suppkey@0 as ps_suppkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: CAST(ps_availqty@1 AS Float64) > __value@2 - ProjectionExec: expr=[ps_suppkey@1 as ps_suppkey, ps_availqty@2 as ps_availqty, __value@5 as __value] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "ps_partkey", index: 0 }, Column { name: "l_partkey", index: 0 }), (Column { name: "ps_suppkey", index: 1 }, Column { name: "l_suppkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }, Column { name: "ps_suppkey", index: 1 }], 4)) - ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, 0.5 * CAST(SUM(lineitem.l_quantity)@2 AS Float64) as __value] - AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }, Column { name: "l_suppkey", index: 1 }], 4)) - -Query Stage #8 (4 -> 4): -ShuffleWriterExec(stage_id=8, output_partitioning=Hash([], 4)) - SortExec: expr=[s_name@0 ASC NULLS LAST] - ProjectionExec: expr=[s_name@1 as s_name, s_address@2 as s_address] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[s_suppkey@1, s_name@2, s_address@3] + ProjectionExec: expr=[n_nationkey@0 as n_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: n_name@1 = KENYA + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, 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)] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_name, s_address, s_nationkey] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "ps_suppkey", index: 0 })] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * sum(lineitem.l_quantity)@1, projection=[ps_suppkey@1] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 4)) - -Query Stage #9 (4 -> 1): -SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] - ShuffleReaderExec(stage_id=8, input_partitioning=Hash([], 4)) + HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(p_partkey@0, ps_partkey@0)] + ProjectionExec: expr=[p_partkey@0 as p_partkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: p_name@1 LIKE blanched% + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/part.parquet]]}, projection=[p_partkey, p_name], predicate=p_name@1 LIKE blanched% + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/partsupp.parquet:0..41091907]]}, projection=[ps_partkey, ps_suppkey, ps_availqty] + ProjectionExec: expr=[0.5 * CAST(sum(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * sum(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] + AggregateExec: mode=Single, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] + ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: l_shipdate@3 >= 1993-01-01 AND l_shipdate@3 < 1994-01-01 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], predicate=l_shipdate@10 >= 1993-01-01 AND l_shipdate@10 < 1994-01-01, pruning_predicate=CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_max@0 >= 1993-01-01 END AND CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_min@3 < 1994-01-01 END, required_guarantees=[] diff --git a/testdata/expected-plans/q21.txt b/testdata/expected-plans/q21.txt index 557f95f..ce41d3d 100644 --- a/testdata/expected-plans/q21.txt +++ b/testdata/expected-plans/q21.txt @@ -3,11 +3,11 @@ DataFusion Logical Plan Limit: skip=0, fetch=100 Sort: numwait DESC NULLS FIRST, supplier.s_name ASC NULLS LAST, fetch=100 - Projection: supplier.s_name, COUNT(UInt8(1)) AS numwait - Aggregate: groupBy=[[supplier.s_name]], aggr=[[COUNT(UInt8(1))]] + Projection: supplier.s_name, count(*) AS numwait + Aggregate: groupBy=[[supplier.s_name]], aggr=[[count(Int64(1)) AS count(*)]] Projection: supplier.s_name - LeftAnti Join: l1.l_orderkey = l3.l_orderkey Filter: l3.l_suppkey != l1.l_suppkey - LeftSemi Join: l1.l_orderkey = l2.l_orderkey Filter: l2.l_suppkey != l1.l_suppkey + LeftAnti Join: l1.l_orderkey = __correlated_sq_2.l_orderkey Filter: __correlated_sq_2.l_suppkey != l1.l_suppkey + LeftSemi Join: l1.l_orderkey = __correlated_sq_1.l_orderkey Filter: __correlated_sq_1.l_suppkey != l1.l_suppkey Projection: supplier.s_name, l1.l_orderkey, l1.l_suppkey Inner Join: supplier.s_nationkey = nation.n_nationkey Projection: supplier.s_name, supplier.s_nationkey, l1.l_orderkey, l1.l_suppkey @@ -25,170 +25,83 @@ Limit: skip=0, fetch=100 Projection: nation.n_nationkey Filter: nation.n_name = Utf8("ARGENTINA") TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("ARGENTINA")] - SubqueryAlias: l2 - TableScan: lineitem projection=[l_orderkey, l_suppkey] - SubqueryAlias: l3 - Projection: lineitem.l_orderkey, lineitem.l_suppkey - Filter: lineitem.l_receiptdate > lineitem.l_commitdate - TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] + SubqueryAlias: __correlated_sq_1 + SubqueryAlias: l2 + TableScan: lineitem projection=[l_orderkey, l_suppkey] + SubqueryAlias: __correlated_sq_2 + SubqueryAlias: l3 + Projection: lineitem.l_orderkey, lineitem.l_suppkey + Filter: lineitem.l_receiptdate > lineitem.l_commitdate + TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] DataFusion Physical Plan ======================== -GlobalLimitExec: skip=0, fetch=100 - SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST] - SortExec: fetch=100, expr=[numwait@1 DESC,s_name@0 ASC NULLS LAST] - ProjectionExec: expr=[s_name@0 as s_name, COUNT(UInt8(1))@1 as numwait] - AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[COUNT(UInt8(1))] +SortExec: TopK(fetch=100), expr=[numwait@1 DESC,s_name@0 ASC NULLS LAST], preserve_partitioning=[false] + ProjectionExec: expr=[s_name@0 as s_name, count(*)@1 as numwait] + AggregateExec: mode=Single, gby=[s_name@0 as s_name], aggr=[count(*)] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_name", index: 0 }], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[COUNT(UInt8(1))] - ProjectionExec: expr=[s_name@0 as s_name] + HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@1)], projection=[s_name@1, l_orderkey@3, l_suppkey@4] + ProjectionExec: expr=[n_nationkey@0 as n_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: n_name@1 = ARGENTINA + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, projection=[n_nationkey, n_name], predicate=n_name@1 = ARGENTINA, pruning_predicate=CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= ARGENTINA AND ARGENTINA <= n_name_max@1 END, required_guarantees=[n_name in (ARGENTINA)] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(Column { name: "l_orderkey", index: 1 }, Column { name: "l_orderkey", index: 0 })], filter=BinaryExpr { left: Column { name: "l_suppkey", index: 1 }, op: NotEq, right: Column { name: "l_suppkey", index: 0 } } - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "l_orderkey", index: 1 }, Column { name: "l_orderkey", index: 0 })], filter=BinaryExpr { left: Column { name: "l_suppkey", index: 1 }, op: NotEq, right: Column { name: "l_suppkey", index: 0 } } - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 1 }], 4), input_partitions=4 - ProjectionExec: expr=[s_name@0 as s_name, l_orderkey@2 as l_orderkey, l_suppkey@3 as l_suppkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 1 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 1 }], 4), input_partitions=4 - ProjectionExec: expr=[s_name@0 as s_name, s_nationkey@1 as s_nationkey, l_orderkey@2 as l_orderkey, l_suppkey@3 as l_suppkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_orderkey", index: 2 }, Column { name: "o_orderkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 2 }], 4), input_partitions=4 - ProjectionExec: expr=[s_name@1 as s_name, s_nationkey@2 as s_nationkey, l_orderkey@3 as l_orderkey, l_suppkey@4 as l_suppkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "l_suppkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_name, s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 4), input_partitions=4 - ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_receiptdate@3 > l_commitdate@2 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_receiptdate@12 > l_commitdate@11, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[o_orderkey@0 as o_orderkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: o_orderstatus@1 = F - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderstatus@2 = F, pruning_predicate=o_orderstatus_min@0 <= F AND F <= o_orderstatus_max@1, projection=[o_orderkey, o_orderstatus] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4 - RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - ProjectionExec: expr=[n_nationkey@0 as n_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: n_name@1 = ARGENTINA - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, predicate=n_name@1 = ARGENTINA, pruning_predicate=n_name_min@0 <= ARGENTINA AND ARGENTINA <= n_name_max@1, projection=[n_nationkey, n_name] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_orderkey, l_suppkey] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@2)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] + ProjectionExec: expr=[o_orderkey@0 as o_orderkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: o_orderstatus@1 = F + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, 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)] CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_name, s_nationkey] ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: l_receiptdate@3 > l_commitdate@2 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_receiptdate@12 > l_commitdate@11, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], predicate=l_receiptdate@12 > l_commitdate@11 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_suppkey] + ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: l_receiptdate@3 > l_commitdate@2 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], predicate=l_receiptdate@12 > l_commitdate@11 RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_name, s_nationkey] - -Query Stage #1 (4 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 4)) - ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_receiptdate@3 > l_commitdate@2 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_receiptdate@12 > l_commitdate@11, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate] - -Query Stage #2 (4 -> 4): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "l_orderkey", index: 2 }], 4)) - ProjectionExec: expr=[s_name@1 as s_name, s_nationkey@2 as s_nationkey, l_orderkey@3 as l_orderkey, l_suppkey@4 as l_suppkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "l_suppkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 4)) - -Query Stage #3 (4 -> 4): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) - ProjectionExec: expr=[o_orderkey@0 as o_orderkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: o_orderstatus@1 = F - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderstatus@2 = F, pruning_predicate=o_orderstatus_min@0 <= F AND F <= o_orderstatus_max@1, projection=[o_orderkey, o_orderstatus] - -Query Stage #4 (4 -> 4): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "s_nationkey", index: 1 }], 4)) - ProjectionExec: expr=[s_name@0 as s_name, s_nationkey@1 as s_nationkey, l_orderkey@2 as l_orderkey, l_suppkey@3 as l_suppkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_orderkey", index: 2 }, Column { name: "o_orderkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "l_orderkey", index: 2 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) - -Query Stage #5 (1 -> 4): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - ProjectionExec: expr=[n_nationkey@0 as n_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: n_name@1 = ARGENTINA - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, predicate=n_name@1 = ARGENTINA, pruning_predicate=n_name_min@0 <= ARGENTINA AND ARGENTINA <= n_name_max@1, projection=[n_nationkey, n_name] - -Query Stage #6 (4 -> 4): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "l_orderkey", index: 1 }], 4)) - ProjectionExec: expr=[s_name@0 as s_name, l_orderkey@2 as l_orderkey, l_suppkey@3 as l_suppkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 1 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "s_nationkey", index: 1 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - -Query Stage #7 (4 -> 4): -ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_orderkey, l_suppkey] - -Query Stage #8 (4 -> 4): -ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_receiptdate@3 > l_commitdate@2 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_receiptdate@12 > l_commitdate@11, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate] - -Query Stage #9 (4 -> 4): -ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "s_name", index: 0 }], 4)) - AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[COUNT(UInt8(1))] - ProjectionExec: expr=[s_name@0 as s_name] +Query Stage #0 (1 -> 1): +SortExec: TopK(fetch=100), expr=[numwait@1 DESC,s_name@0 ASC NULLS LAST], preserve_partitioning=[false] + ProjectionExec: expr=[s_name@0 as s_name, count(*)@1 as numwait] + AggregateExec: mode=Single, gby=[s_name@0 as s_name], aggr=[count(*)] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(Column { name: "l_orderkey", index: 1 }, Column { name: "l_orderkey", index: 0 })], filter=BinaryExpr { left: Column { name: "l_suppkey", index: 1 }, op: NotEq, right: Column { name: "l_suppkey", index: 0 } } + HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "l_orderkey", index: 1 }, Column { name: "l_orderkey", index: 0 })], filter=BinaryExpr { left: Column { name: "l_suppkey", index: 1 }, op: NotEq, right: Column { name: "l_suppkey", index: 0 } } - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "l_orderkey", index: 1 }], 4)) + HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - -Query Stage #10 (4 -> 4): -ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "s_name", index: 0 }], 4)) - SortExec: fetch=100, expr=[numwait@1 DESC,s_name@0 ASC NULLS LAST] - ProjectionExec: expr=[s_name@0 as s_name, COUNT(UInt8(1))@1 as numwait] - AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[COUNT(UInt8(1))] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "s_name", index: 0 }], 4)) - -Query Stage #11 (1 -> 1): -GlobalLimitExec: skip=0, fetch=100 - SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST] - ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "s_name", index: 0 }], 4)) + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@1)], projection=[s_name@1, l_orderkey@3, l_suppkey@4] + ProjectionExec: expr=[n_nationkey@0 as n_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: n_name@1 = ARGENTINA + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, projection=[n_nationkey, n_name], predicate=n_name@1 = ARGENTINA, pruning_predicate=CASE WHEN n_name_null_count@2 = n_name_row_count@3 THEN false ELSE n_name_min@0 <= ARGENTINA AND ARGENTINA <= n_name_max@1 END, required_guarantees=[n_name in (ARGENTINA)] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@2)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] + ProjectionExec: expr=[o_orderkey@0 as o_orderkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: o_orderstatus@1 = F + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, 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)] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_name, s_nationkey] + ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: l_receiptdate@3 > l_commitdate@2 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], predicate=l_receiptdate@12 > l_commitdate@11 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_suppkey] + ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: l_receiptdate@3 > l_commitdate@2 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], predicate=l_receiptdate@12 > l_commitdate@11 diff --git a/testdata/expected-plans/q22.txt b/testdata/expected-plans/q22.txt index cdc613d..04c8a49 100644 --- a/testdata/expected-plans/q22.txt +++ b/testdata/expected-plans/q22.txt @@ -2,113 +2,61 @@ DataFusion Logical Plan ======================= Sort: custsale.cntrycode ASC NULLS LAST - Projection: custsale.cntrycode, COUNT(UInt8(1)) AS numcust, SUM(custsale.c_acctbal) AS totacctbal - Aggregate: groupBy=[[custsale.cntrycode]], aggr=[[COUNT(UInt8(1)), SUM(custsale.c_acctbal)]] + Projection: custsale.cntrycode, count(*) AS numcust, sum(custsale.c_acctbal) AS totacctbal + Aggregate: groupBy=[[custsale.cntrycode]], aggr=[[count(Int64(1)) AS count(*), sum(custsale.c_acctbal)]] SubqueryAlias: custsale Projection: substr(customer.c_phone, Int64(1), Int64(2)) AS cntrycode, customer.c_acctbal - Filter: CAST(customer.c_acctbal AS Decimal128(15, 6)) > __scalar_sq_1.__value - CrossJoin: - Projection: customer.c_phone, customer.c_acctbal - LeftAnti Join: customer.c_custkey = orders.o_custkey - Filter: substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("24"), Utf8("34"), Utf8("16"), Utf8("30"), Utf8("33"), Utf8("14"), Utf8("13")]) - TableScan: customer projection=[c_custkey, c_phone, c_acctbal], partial_filters=[substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("24"), Utf8("34"), Utf8("16"), Utf8("30"), Utf8("33"), Utf8("14"), Utf8("13")])] + Inner Join: Filter: CAST(customer.c_acctbal AS Decimal128(15, 6)) > __scalar_sq_2.avg(customer.c_acctbal) + Projection: customer.c_phone, customer.c_acctbal + LeftAnti Join: customer.c_custkey = __correlated_sq_1.o_custkey + Filter: substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("24"), Utf8("34"), Utf8("16"), Utf8("30"), Utf8("33"), Utf8("14"), Utf8("13")]) + TableScan: customer projection=[c_custkey, c_phone, c_acctbal], partial_filters=[substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("24"), Utf8("34"), Utf8("16"), Utf8("30"), Utf8("33"), Utf8("14"), Utf8("13")])] + SubqueryAlias: __correlated_sq_1 TableScan: orders projection=[o_custkey] - SubqueryAlias: __scalar_sq_1 - Projection: AVG(customer.c_acctbal) AS __value - Aggregate: groupBy=[[]], aggr=[[AVG(customer.c_acctbal)]] - Projection: customer.c_acctbal - Filter: customer.c_acctbal > Decimal128(Some(0),11,2) AND substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("24"), Utf8("34"), Utf8("16"), Utf8("30"), Utf8("33"), Utf8("14"), Utf8("13")]) - TableScan: customer projection=[c_phone, c_acctbal], partial_filters=[customer.c_acctbal > Decimal128(Some(0),11,2) AS customer.c_acctbal > Decimal128(Some(0),30,15), substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("24"), Utf8("34"), Utf8("16"), Utf8("30"), Utf8("33"), Utf8("14"), Utf8("13")]), customer.c_acctbal > Decimal128(Some(0),11,2)] + SubqueryAlias: __scalar_sq_2 + Aggregate: groupBy=[[]], aggr=[[avg(customer.c_acctbal)]] + Projection: customer.c_acctbal + Filter: customer.c_acctbal > Decimal128(Some(0),11,2) AND substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("24"), Utf8("34"), Utf8("16"), Utf8("30"), Utf8("33"), Utf8("14"), Utf8("13")]) + TableScan: customer projection=[c_phone, c_acctbal], partial_filters=[customer.c_acctbal > Decimal128(Some(0),11,2) AS customer.c_acctbal > Decimal128(Some(0),30,15), substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("24"), Utf8("34"), Utf8("16"), Utf8("30"), Utf8("33"), Utf8("14"), Utf8("13")]), customer.c_acctbal > Decimal128(Some(0),11,2)] DataFusion Physical Plan ======================== -SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] - SortExec: expr=[cntrycode@0 ASC NULLS LAST] - ProjectionExec: expr=[cntrycode@0 as cntrycode, COUNT(UInt8(1))@1 as numcust, SUM(custsale.c_acctbal)@2 as totacctbal] - AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(UInt8(1)), SUM(custsale.c_acctbal)] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "cntrycode", index: 0 }], 4), input_partitions=1 - AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(UInt8(1)), SUM(custsale.c_acctbal)] - ProjectionExec: expr=[substr(c_phone@0, 1, 2) as cntrycode, c_acctbal@1 as c_acctbal] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: CAST(c_acctbal@1 AS Decimal128(15, 6)) > __value@2 - CrossJoinExec - CoalescePartitionsExec - ProjectionExec: expr=[c_phone@1 as c_phone, c_acctbal@2 as c_acctbal] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4), input_partitions=4 - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: Use substr(c_phone@1, 1, 2) IN (SET) ([Literal { value: Utf8("24") }, Literal { value: Utf8("34") }, Literal { value: Utf8("16") }, Literal { value: Utf8("30") }, Literal { value: Utf8("33") }, Literal { value: Utf8("14") }, Literal { value: Utf8("13") }]) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, predicate=Use substr(c_phone@4, 1, 2) IN (SET) ([Literal { value: Utf8("24") }, Literal { value: Utf8("34") }, Literal { value: Utf8("16") }, Literal { value: Utf8("30") }, Literal { value: Utf8("33") }, Literal { value: Utf8("14") }, Literal { value: Utf8("13") }]), projection=[c_custkey, c_phone, c_acctbal] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_custkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, projection=[o_custkey] - ProjectionExec: expr=[AVG(customer.c_acctbal)@0 as __value] - AggregateExec: mode=Final, gby=[], aggr=[AVG(customer.c_acctbal)] - CoalescePartitionsExec - AggregateExec: mode=Partial, gby=[], aggr=[AVG(customer.c_acctbal)] - ProjectionExec: expr=[c_acctbal@1 as c_acctbal] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: c_acctbal@1 > Some(0),11,2 AND Use substr(c_phone@0, 1, 2) IN (SET) ([Literal { value: Utf8("24") }, Literal { value: Utf8("34") }, Literal { value: Utf8("16") }, Literal { value: Utf8("30") }, Literal { value: Utf8("33") }, Literal { value: Utf8("14") }, Literal { value: Utf8("13") }]) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, predicate=c_acctbal@5 > Some(0),11,2 AND Use substr(c_phone@4, 1, 2) IN (SET) ([Literal { value: Utf8("24") }, Literal { value: Utf8("34") }, Literal { value: Utf8("16") }, Literal { value: Utf8("30") }, Literal { value: Utf8("33") }, Literal { value: Utf8("14") }, Literal { value: Utf8("13") }]) AND c_acctbal@5 > Some(0),11,2, pruning_predicate=c_acctbal_max@0 > Some(0),11,2 AND c_acctbal_max@0 > Some(0),11,2, projection=[c_phone, c_acctbal] +SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[false] + ProjectionExec: expr=[cntrycode@0 as cntrycode, count(*)@1 as numcust, sum(custsale.c_acctbal)@2 as totacctbal] + AggregateExec: mode=Single, gby=[cntrycode@0 as cntrycode], aggr=[count(*), sum(custsale.c_acctbal)] + ProjectionExec: expr=[substr(c_phone@1, 1, 2) as cntrycode, c_acctbal@2 as c_acctbal] + NestedLoopJoinExec: join_type=Inner, filter=CAST(c_acctbal@0 AS Decimal128(15, 6)) > avg(customer.c_acctbal)@1 + AggregateExec: mode=Single, gby=[], aggr=[avg(customer.c_acctbal)] + ProjectionExec: expr=[c_acctbal@1 as c_acctbal] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: c_acctbal@1 > Some(0),11,2 AND Use substr(c_phone@0, 1, 2) IN (SET) ([Literal { value: Utf8("24") }, Literal { value: Utf8("34") }, Literal { value: Utf8("16") }, Literal { value: Utf8("30") }, Literal { value: Utf8("33") }, Literal { value: Utf8("14") }, Literal { value: Utf8("13") }]) + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/customer.parquet:0..13211178]]}, projection=[c_phone, c_acctbal], predicate=c_acctbal@5 > Some(0),11,2 AND Use substr(c_phone@4, 1, 2) IN (SET) ([Literal { value: Utf8("24") }, Literal { value: Utf8("34") }, Literal { value: Utf8("16") }, Literal { value: Utf8("30") }, Literal { value: Utf8("33") }, Literal { value: Utf8("14") }, Literal { value: Utf8("13") }]) AND c_acctbal@5 > Some(0),11,2, pruning_predicate=CASE WHEN c_acctbal_null_count@1 = c_acctbal_row_count@2 THEN false ELSE c_acctbal_max@0 > Some(0),11,2 END AND CASE WHEN c_acctbal_null_count@1 = c_acctbal_row_count@2 THEN false ELSE c_acctbal_max@0 > Some(0),11,2 END, required_guarantees=[] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: Use substr(c_phone@1, 1, 2) IN (SET) ([Literal { value: Utf8("24") }, Literal { value: Utf8("34") }, Literal { value: Utf8("16") }, Literal { value: Utf8("30") }, Literal { value: Utf8("33") }, Literal { value: Utf8("14") }, Literal { value: Utf8("13") }]) + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/customer.parquet:0..13211178]]}, projection=[c_custkey, c_phone, c_acctbal], predicate=Use substr(c_phone@4, 1, 2) IN (SET) ([Literal { value: Utf8("24") }, Literal { value: Utf8("34") }, Literal { value: Utf8("16") }, Literal { value: Utf8("30") }, Literal { value: Utf8("33") }, Literal { value: Utf8("14") }, Literal { value: Utf8("13") }]) + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_custkey] RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: Use substr(c_phone@1, 1, 2) IN (SET) ([Literal { value: Utf8("24") }, Literal { value: Utf8("34") }, Literal { value: Utf8("16") }, Literal { value: Utf8("30") }, Literal { value: Utf8("33") }, Literal { value: Utf8("14") }, Literal { value: Utf8("13") }]) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, predicate=Use substr(c_phone@4, 1, 2) IN (SET) ([Literal { value: Utf8("24") }, Literal { value: Utf8("34") }, Literal { value: Utf8("16") }, Literal { value: Utf8("30") }, Literal { value: Utf8("33") }, Literal { value: Utf8("14") }, Literal { value: Utf8("13") }]), projection=[c_custkey, c_phone, c_acctbal] - -Query Stage #1 (4 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_custkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, projection=[o_custkey] - -Query Stage #2 (4 -> 4): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([], 4)) - ProjectionExec: expr=[c_phone@1 as c_phone, c_acctbal@2 as c_acctbal] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_custkey", index: 0 }], 4)) - -Query Stage #3 (4 -> 1): -ShuffleWriterExec(stage_id=3, output_partitioning=UnknownPartitioning(4)) - AggregateExec: mode=Partial, gby=[], aggr=[AVG(customer.c_acctbal)] - ProjectionExec: expr=[c_acctbal@1 as c_acctbal] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: c_acctbal@1 > Some(0),11,2 AND Use substr(c_phone@0, 1, 2) IN (SET) ([Literal { value: Utf8("24") }, Literal { value: Utf8("34") }, Literal { value: Utf8("16") }, Literal { value: Utf8("30") }, Literal { value: Utf8("33") }, Literal { value: Utf8("14") }, Literal { value: Utf8("13") }]) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, predicate=c_acctbal@5 > Some(0),11,2 AND Use substr(c_phone@4, 1, 2) IN (SET) ([Literal { value: Utf8("24") }, Literal { value: Utf8("34") }, Literal { value: Utf8("16") }, Literal { value: Utf8("30") }, Literal { value: Utf8("33") }, Literal { value: Utf8("14") }, Literal { value: Utf8("13") }]) AND c_acctbal@5 > Some(0),11,2, pruning_predicate=c_acctbal_max@0 > Some(0),11,2 AND c_acctbal_max@0 > Some(0),11,2, projection=[c_phone, c_acctbal] - -Query Stage #4 (1 -> 4): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "cntrycode", index: 0 }], 4)) - AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(UInt8(1)), SUM(custsale.c_acctbal)] - ProjectionExec: expr=[substr(c_phone@0, 1, 2) as cntrycode, c_acctbal@1 as c_acctbal] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: CAST(c_acctbal@1 AS Decimal128(15, 6)) > __value@2 - CrossJoinExec - CoalescePartitionsExec - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([], 4)) - ProjectionExec: expr=[AVG(customer.c_acctbal)@0 as __value] - AggregateExec: mode=Final, gby=[], aggr=[AVG(customer.c_acctbal)] - CoalescePartitionsExec - ShuffleReaderExec(stage_id=3, input_partitioning=UnknownPartitioning(4)) - -Query Stage #5 (4 -> 4): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "cntrycode", index: 0 }], 4)) - SortExec: expr=[cntrycode@0 ASC NULLS LAST] - ProjectionExec: expr=[cntrycode@0 as cntrycode, COUNT(UInt8(1))@1 as numcust, SUM(custsale.c_acctbal)@2 as totacctbal] - AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(UInt8(1)), SUM(custsale.c_acctbal)] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "cntrycode", index: 0 }], 4)) - -Query Stage #6 (4 -> 1): -SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "cntrycode", index: 0 }], 4)) +Query Stage #0 (1 -> 1): +SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[false] + ProjectionExec: expr=[cntrycode@0 as cntrycode, count(*)@1 as numcust, sum(custsale.c_acctbal)@2 as totacctbal] + AggregateExec: mode=Single, gby=[cntrycode@0 as cntrycode], aggr=[count(*), sum(custsale.c_acctbal)] + ProjectionExec: expr=[substr(c_phone@1, 1, 2) as cntrycode, c_acctbal@2 as c_acctbal] + NestedLoopJoinExec: join_type=Inner, filter=CAST(c_acctbal@0 AS Decimal128(15, 6)) > avg(customer.c_acctbal)@1 + AggregateExec: mode=Single, gby=[], aggr=[avg(customer.c_acctbal)] + ProjectionExec: expr=[c_acctbal@1 as c_acctbal] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: c_acctbal@1 > Some(0),11,2 AND Use substr(c_phone@0, 1, 2) IN (SET) ([Literal { value: Utf8("24") }, Literal { value: Utf8("34") }, Literal { value: Utf8("16") }, Literal { value: Utf8("30") }, Literal { value: Utf8("33") }, Literal { value: Utf8("14") }, Literal { value: Utf8("13") }]) + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/customer.parquet:0..13211178]]}, projection=[c_phone, c_acctbal], predicate=c_acctbal@5 > Some(0),11,2 AND Use substr(c_phone@4, 1, 2) IN (SET) ([Literal { value: Utf8("24") }, Literal { value: Utf8("34") }, Literal { value: Utf8("16") }, Literal { value: Utf8("30") }, Literal { value: Utf8("33") }, Literal { value: Utf8("14") }, Literal { value: Utf8("13") }]) AND c_acctbal@5 > Some(0),11,2, pruning_predicate=CASE WHEN c_acctbal_null_count@1 = c_acctbal_row_count@2 THEN false ELSE c_acctbal_max@0 > Some(0),11,2 END AND CASE WHEN c_acctbal_null_count@1 = c_acctbal_row_count@2 THEN false ELSE c_acctbal_max@0 > Some(0),11,2 END, required_guarantees=[] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: Use substr(c_phone@1, 1, 2) IN (SET) ([Literal { value: Utf8("24") }, Literal { value: Utf8("34") }, Literal { value: Utf8("16") }, Literal { value: Utf8("30") }, Literal { value: Utf8("33") }, Literal { value: Utf8("14") }, Literal { value: Utf8("13") }]) + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/customer.parquet:0..13211178]]}, projection=[c_custkey, c_phone, c_acctbal], predicate=Use substr(c_phone@4, 1, 2) IN (SET) ([Literal { value: Utf8("24") }, Literal { value: Utf8("34") }, Literal { value: Utf8("16") }, Literal { value: Utf8("30") }, Literal { value: Utf8("33") }, Literal { value: Utf8("14") }, Literal { value: Utf8("13") }]) + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_custkey] diff --git a/testdata/expected-plans/q3.txt b/testdata/expected-plans/q3.txt index e6f9749..84f91b2 100644 --- a/testdata/expected-plans/q3.txt +++ b/testdata/expected-plans/q3.txt @@ -3,8 +3,8 @@ DataFusion Logical Plan Limit: skip=0, fetch=10 Sort: revenue DESC NULLS FIRST, orders.o_orderdate ASC NULLS LAST, fetch=10 - Projection: lineitem.l_orderkey, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue, orders.o_orderdate, orders.o_shippriority - Aggregate: groupBy=[[lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority]], aggr=[[SUM(CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] + Projection: lineitem.l_orderkey, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue, orders.o_orderdate, orders.o_shippriority + Aggregate: groupBy=[[lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority]], aggr=[[sum(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] Projection: orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount Inner Join: orders.o_orderkey = lineitem.l_orderkey Projection: orders.o_orderkey, orders.o_orderdate, orders.o_shippriority @@ -12,103 +12,54 @@ Limit: skip=0, fetch=10 Projection: customer.c_custkey Filter: customer.c_mktsegment = Utf8("BUILDING") TableScan: customer projection=[c_custkey, c_mktsegment], partial_filters=[customer.c_mktsegment = Utf8("BUILDING")] - Filter: orders.o_orderdate < Date32("9204") - TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], partial_filters=[orders.o_orderdate < Date32("9204")] + Filter: orders.o_orderdate < Date32("1995-03-15") + TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], partial_filters=[orders.o_orderdate < Date32("1995-03-15")] Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount - Filter: lineitem.l_shipdate > Date32("9204") - TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate > Date32("9204")] + Filter: lineitem.l_shipdate > Date32("1995-03-15") + TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate > Date32("1995-03-15")] DataFusion Physical Plan ======================== -GlobalLimitExec: skip=0, fetch=10 - SortPreservingMergeExec: [revenue@1 DESC,o_orderdate@2 ASC NULLS LAST] - SortExec: fetch=10, expr=[revenue@1 DESC,o_orderdate@2 ASC NULLS LAST] - ProjectionExec: expr=[l_orderkey@0 as l_orderkey, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] - AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +SortExec: TopK(fetch=10), expr=[revenue@1 DESC,o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[false] + ProjectionExec: expr=[l_orderkey@0 as l_orderkey, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] + AggregateExec: mode=Single, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, o_shippriority@2, l_orderkey@3, l_extendedprice@4, l_discount@5] CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 1 }, Column { name: "o_shippriority", index: 2 }], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - ProjectionExec: expr=[o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority, l_orderkey@3 as l_orderkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 0 }, Column { name: "l_orderkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[o_orderkey@1 as o_orderkey, o_orderdate@3 as o_orderdate, o_shippriority@4 as o_shippriority] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[c_custkey@0 as c_custkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: c_mktsegment@1 = BUILDING - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, predicate=c_mktsegment@6 = BUILDING, pruning_predicate=c_mktsegment_min@0 <= BUILDING AND BUILDING <= c_mktsegment_max@1, projection=[c_custkey, c_mktsegment] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4), input_partitions=4 - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: o_orderdate@2 < 9204 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderdate@4 < 9204, pruning_predicate=o_orderdate_min@0 < 9204, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_shipdate@3 > 9204 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate@10 > 9204, pruning_predicate=l_shipdate_max@0 > 9204, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] + ProjectionExec: expr=[c_custkey@0 as c_custkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: c_mktsegment@1 = BUILDING + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/customer.parquet:0..13211178]]}, projection=[c_custkey, c_mktsegment], predicate=c_mktsegment@6 = BUILDING, pruning_predicate=CASE WHEN c_mktsegment_null_count@2 = c_mktsegment_row_count@3 THEN false ELSE c_mktsegment_min@0 <= BUILDING AND BUILDING <= c_mktsegment_max@1 END, required_guarantees=[c_mktsegment in (BUILDING)] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: o_orderdate@2 < 1995-03-15 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], predicate=o_orderdate@4 < 1995-03-15, pruning_predicate=CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_min@0 < 1995-03-15 END, required_guarantees=[] + ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: l_shipdate@3 > 1995-03-15 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], predicate=l_shipdate@10 > 1995-03-15, pruning_predicate=CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_max@0 > 1995-03-15 END, required_guarantees=[] RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) - ProjectionExec: expr=[c_custkey@0 as c_custkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: c_mktsegment@1 = BUILDING - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, predicate=c_mktsegment@6 = BUILDING, pruning_predicate=c_mktsegment_min@0 <= BUILDING AND BUILDING <= c_mktsegment_max@1, projection=[c_custkey, c_mktsegment] - -Query Stage #1 (4 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: o_orderdate@2 < 9204 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderdate@4 < 9204, pruning_predicate=o_orderdate_min@0 < 9204, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority] - -Query Stage #2 (4 -> 4): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) - ProjectionExec: expr=[o_orderkey@1 as o_orderkey, o_orderdate@3 as o_orderdate, o_shippriority@4 as o_shippriority] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) - -Query Stage #3 (4 -> 4): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_shipdate@3 > 9204 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate@10 > 9204, pruning_predicate=l_shipdate_max@0 > 9204, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate] - -Query Stage #4 (4 -> 4): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 1 }, Column { name: "o_shippriority", index: 2 }], 4)) - AggregateExec: mode=Partial, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - ProjectionExec: expr=[o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority, l_orderkey@3 as l_orderkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] +Query Stage #0 (1 -> 1): +SortExec: TopK(fetch=10), expr=[revenue@1 DESC,o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[false] + ProjectionExec: expr=[l_orderkey@0 as l_orderkey, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] + AggregateExec: mode=Single, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 0 }, Column { name: "l_orderkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, o_shippriority@2, l_orderkey@3, l_extendedprice@4, l_discount@5] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - -Query Stage #5 (4 -> 4): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 2 }, Column { name: "o_shippriority", index: 3 }], 4)) - SortExec: fetch=10, expr=[revenue@1 DESC,o_orderdate@2 ASC NULLS LAST] - ProjectionExec: expr=[l_orderkey@0 as l_orderkey, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] - AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - 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 }], 4)) - -Query Stage #6 (1 -> 1): -GlobalLimitExec: skip=0, fetch=10 - SortPreservingMergeExec: [revenue@1 DESC,o_orderdate@2 ASC NULLS LAST] - 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 }], 4)) + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] + ProjectionExec: expr=[c_custkey@0 as c_custkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: c_mktsegment@1 = BUILDING + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/customer.parquet:0..13211178]]}, projection=[c_custkey, c_mktsegment], predicate=c_mktsegment@6 = BUILDING, pruning_predicate=CASE WHEN c_mktsegment_null_count@2 = c_mktsegment_row_count@3 THEN false ELSE c_mktsegment_min@0 <= BUILDING AND BUILDING <= c_mktsegment_max@1 END, required_guarantees=[c_mktsegment in (BUILDING)] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: o_orderdate@2 < 1995-03-15 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], predicate=o_orderdate@4 < 1995-03-15, pruning_predicate=CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_min@0 < 1995-03-15 END, required_guarantees=[] + ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: l_shipdate@3 > 1995-03-15 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], predicate=l_shipdate@10 > 1995-03-15, pruning_predicate=CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_max@0 > 1995-03-15 END, required_guarantees=[] diff --git a/testdata/expected-plans/q4.txt b/testdata/expected-plans/q4.txt index 17a19e1..8dc100a 100644 --- a/testdata/expected-plans/q4.txt +++ b/testdata/expected-plans/q4.txt @@ -2,80 +2,50 @@ DataFusion Logical Plan ======================= Sort: orders.o_orderpriority ASC NULLS LAST - Projection: orders.o_orderpriority, COUNT(UInt8(1)) AS order_count - Aggregate: groupBy=[[orders.o_orderpriority]], aggr=[[COUNT(UInt8(1))]] + Projection: orders.o_orderpriority, count(*) AS order_count + Aggregate: groupBy=[[orders.o_orderpriority]], aggr=[[count(Int64(1)) AS count(*)]] Projection: orders.o_orderpriority - LeftSemi Join: orders.o_orderkey = lineitem.l_orderkey + LeftSemi Join: orders.o_orderkey = __correlated_sq_1.l_orderkey Projection: orders.o_orderkey, orders.o_orderpriority - Filter: orders.o_orderdate >= Date32("9221") AND orders.o_orderdate < Date32("9312") - TableScan: orders projection=[o_orderkey, o_orderdate, o_orderpriority], partial_filters=[orders.o_orderdate >= Date32("9221"), orders.o_orderdate < Date32("9312")] - Projection: lineitem.l_orderkey - Filter: lineitem.l_commitdate < lineitem.l_receiptdate - TableScan: lineitem projection=[l_orderkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_commitdate < lineitem.l_receiptdate] + Filter: orders.o_orderdate >= Date32("1995-04-01") AND orders.o_orderdate < Date32("1995-07-01") + TableScan: orders projection=[o_orderkey, o_orderdate, o_orderpriority], partial_filters=[orders.o_orderdate >= Date32("1995-04-01"), orders.o_orderdate < Date32("1995-07-01")] + SubqueryAlias: __correlated_sq_1 + Projection: lineitem.l_orderkey + Filter: lineitem.l_receiptdate > lineitem.l_commitdate + TableScan: lineitem projection=[l_orderkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] DataFusion Physical Plan ======================== -SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] - SortExec: expr=[o_orderpriority@0 ASC NULLS LAST] - ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, COUNT(UInt8(1))@1 as order_count] - AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(UInt8(1))] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(UInt8(1))] - ProjectionExec: expr=[o_orderpriority@1 as o_orderpriority] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "o_orderkey", index: 0 }, Column { name: "l_orderkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: o_orderdate@1 >= 9221 AND o_orderdate@1 < 9312 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderdate@4 >= 9221 AND o_orderdate@4 < 9312, pruning_predicate=o_orderdate_max@0 >= 9221 AND o_orderdate_min@1 < 9312, projection=[o_orderkey, o_orderdate, o_orderpriority] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[l_orderkey@0 as l_orderkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_commitdate@1 < l_receiptdate@2 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_commitdate@11 < l_receiptdate@12, projection=[l_orderkey, l_commitdate, l_receiptdate] +SortExec: expr=[o_orderpriority@0 ASC NULLS LAST], preserve_partitioning=[false] + ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, count(*)@1 as order_count] + AggregateExec: mode=Single, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(*)] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] + ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: o_orderdate@1 >= 1995-04-01 AND o_orderdate@1 < 1995-07-01 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_orderkey, o_orderdate, o_orderpriority], predicate=o_orderdate@4 >= 1995-04-01 AND o_orderdate@4 < 1995-07-01, pruning_predicate=CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_max@0 >= 1995-04-01 END AND CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_min@3 < 1995-07-01 END, required_guarantees=[] + ProjectionExec: expr=[l_orderkey@0 as l_orderkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: l_receiptdate@2 > l_commitdate@1 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_commitdate, l_receiptdate], predicate=l_receiptdate@12 > l_commitdate@11 RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) - ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: o_orderdate@1 >= 9221 AND o_orderdate@1 < 9312 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderdate@4 >= 9221 AND o_orderdate@4 < 9312, pruning_predicate=o_orderdate_max@0 >= 9221 AND o_orderdate_min@1 < 9312, projection=[o_orderkey, o_orderdate, o_orderpriority] - -Query Stage #1 (4 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - ProjectionExec: expr=[l_orderkey@0 as l_orderkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_commitdate@1 < l_receiptdate@2 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_commitdate@11 < l_receiptdate@12, projection=[l_orderkey, l_commitdate, l_receiptdate] - -Query Stage #2 (4 -> 4): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 4)) - AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(UInt8(1))] - ProjectionExec: expr=[o_orderpriority@1 as o_orderpriority] +Query Stage #0 (1 -> 1): +SortExec: expr=[o_orderpriority@0 ASC NULLS LAST], preserve_partitioning=[false] + ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, count(*)@1 as order_count] + AggregateExec: mode=Single, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(*)] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "o_orderkey", index: 0 }, Column { name: "l_orderkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - -Query Stage #3 (4 -> 4): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 4)) - SortExec: expr=[o_orderpriority@0 ASC NULLS LAST] - ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, COUNT(UInt8(1))@1 as order_count] - AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(UInt8(1))] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 4)) - -Query Stage #4 (4 -> 1): -SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 4)) + HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] + ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: o_orderdate@1 >= 1995-04-01 AND o_orderdate@1 < 1995-07-01 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_orderkey, o_orderdate, o_orderpriority], predicate=o_orderdate@4 >= 1995-04-01 AND o_orderdate@4 < 1995-07-01, pruning_predicate=CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_max@0 >= 1995-04-01 END AND CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_min@3 < 1995-07-01 END, required_guarantees=[] + ProjectionExec: expr=[l_orderkey@0 as l_orderkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: l_receiptdate@2 > l_commitdate@1 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_commitdate, l_receiptdate], predicate=l_receiptdate@12 > l_commitdate@11 diff --git a/testdata/expected-plans/q5.txt b/testdata/expected-plans/q5.txt index ca7997e..ec776aa 100644 --- a/testdata/expected-plans/q5.txt +++ b/testdata/expected-plans/q5.txt @@ -2,8 +2,8 @@ DataFusion Logical Plan ======================= Sort: revenue DESC NULLS FIRST - Projection: nation.n_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue - Aggregate: groupBy=[[nation.n_name]], aggr=[[SUM(CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] + Projection: nation.n_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue + Aggregate: groupBy=[[nation.n_name]], 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, nation.n_name Inner Join: nation.n_regionkey = region.r_regionkey Projection: lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_regionkey @@ -16,8 +16,8 @@ Sort: revenue DESC NULLS FIRST Inner Join: customer.c_custkey = orders.o_custkey TableScan: customer projection=[c_custkey, c_nationkey] Projection: orders.o_orderkey, orders.o_custkey - Filter: orders.o_orderdate >= Date32("8766") AND orders.o_orderdate < Date32("9131") - TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("8766"), orders.o_orderdate < Date32("9131")] + Filter: orders.o_orderdate >= Date32("1994-01-01") AND orders.o_orderdate < Date32("1995-01-01") + TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("1994-01-01"), orders.o_orderdate < Date32("1995-01-01")] TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount] TableScan: supplier projection=[s_suppkey, s_nationkey] TableScan: nation projection=[n_nationkey, n_name, n_regionkey] @@ -28,155 +28,63 @@ Sort: revenue DESC NULLS FIRST DataFusion Physical Plan ======================== -SortPreservingMergeExec: [revenue@1 DESC] - SortExec: expr=[revenue@1 DESC] - ProjectionExec: expr=[n_name@0 as n_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue] - AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "n_name", index: 0 }], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, n_name@2 as n_name] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "n_regionkey", index: 3 }, Column { name: "r_regionkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "n_regionkey", index: 3 }], 4), input_partitions=4 - ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, n_name@4 as n_name, n_regionkey@5 as n_regionkey] +SortExec: expr=[revenue@1 DESC], preserve_partitioning=[false] + ProjectionExec: expr=[n_name@0 as n_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue] + AggregateExec: mode=Single, gby=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, n_name@3] + ProjectionExec: expr=[r_regionkey@0 as r_regionkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: r_name@1 = AFRICA + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/region.parquet]]}, 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)] + ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, n_name@0 as n_name, n_regionkey@1 as n_regionkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_name@1, n_regionkey@2, l_extendedprice@3, l_discount@4] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, projection=[n_nationkey, n_name, n_regionkey] + ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@0 as s_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1), (s_nationkey@1, c_nationkey@0)], projection=[s_nationkey@1, l_extendedprice@4, l_discount@5] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 2 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 4), input_partitions=4 - ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@5 as s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_suppkey", index: 1 }, Column { name: "s_suppkey", index: 0 }), (Column { name: "c_nationkey", index: 0 }, Column { name: "s_nationkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_suppkey", index: 1 }, Column { name: "c_nationkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[c_nationkey@0 as c_nationkey, l_suppkey@3 as l_suppkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 1 }, Column { name: "l_orderkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 1 }], 4), input_partitions=4 - ProjectionExec: expr=[c_nationkey@1 as c_nationkey, o_orderkey@2 as o_orderkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, projection=[c_custkey, c_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4), input_partitions=4 - ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: o_orderdate@2 >= 8766 AND o_orderdate@2 < 9131 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderdate@4 >= 8766 AND o_orderdate@4 < 9131, pruning_predicate=o_orderdate_max@0 >= 8766 AND o_orderdate_min@1 < 9131, projection=[o_orderkey, o_custkey, o_orderdate] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }, Column { name: "s_nationkey", index: 1 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=1 - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_name, n_regionkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4), input_partitions=4 - RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - ProjectionExec: expr=[r_regionkey@0 as r_regionkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: r_name@1 = AFRICA - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/region.parquet/part-0.parquet]]}, predicate=r_name@1 = AFRICA, pruning_predicate=r_name_min@0 <= AFRICA AND AFRICA <= r_name_max@1, projection=[r_regionkey, r_name] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/customer.parquet:0..13211178]]}, projection=[c_custkey, c_nationkey] + ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_orderkey, o_custkey, o_orderdate], predicate=o_orderdate@4 >= 1994-01-01 AND o_orderdate@4 < 1995-01-01, pruning_predicate=CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_max@0 >= 1994-01-01 END AND CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_min@3 < 1995-01-01 END, required_guarantees=[] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount] RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, projection=[c_custkey, c_nationkey] - -Query Stage #1 (4 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) - ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: o_orderdate@2 >= 8766 AND o_orderdate@2 < 9131 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderdate@4 >= 8766 AND o_orderdate@4 < 9131, pruning_predicate=o_orderdate_max@0 >= 8766 AND o_orderdate_min@1 < 9131, projection=[o_orderkey, o_custkey, o_orderdate] - -Query Stage #2 (4 -> 4): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_orderkey", index: 1 }], 4)) - ProjectionExec: expr=[c_nationkey@1 as c_nationkey, o_orderkey@2 as o_orderkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) - -Query Stage #3 (4 -> 4): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount] - -Query Stage #4 (4 -> 4): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_suppkey", index: 1 }, Column { name: "c_nationkey", index: 0 }], 4)) - ProjectionExec: expr=[c_nationkey@0 as c_nationkey, l_suppkey@3 as l_suppkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 1 }, Column { name: "l_orderkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_orderkey", index: 1 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - -Query Stage #5 (4 -> 4): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }, Column { name: "s_nationkey", index: 1 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] - -Query Stage #6 (4 -> 4): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 4)) - ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@5 as s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_suppkey", index: 1 }, Column { name: "s_suppkey", index: 0 }), (Column { name: "c_nationkey", index: 0 }, Column { name: "s_nationkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_suppkey", index: 1 }, Column { name: "c_nationkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }, Column { name: "s_nationkey", index: 1 }], 4)) - -Query Stage #7 (1 -> 4): -ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_name, n_regionkey] - -Query Stage #8 (4 -> 4): -ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "n_regionkey", index: 3 }], 4)) - ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, n_name@4 as n_name, n_regionkey@5 as n_regionkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 2 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - -Query Stage #9 (1 -> 4): -ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4)) - ProjectionExec: expr=[r_regionkey@0 as r_regionkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: r_name@1 = AFRICA - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/region.parquet/part-0.parquet]]}, predicate=r_name@1 = AFRICA, pruning_predicate=r_name_min@0 <= AFRICA AND AFRICA <= r_name_max@1, projection=[r_regionkey, r_name] - -Query Stage #10 (4 -> 4): -ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "n_name", index: 0 }], 4)) - AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, n_name@2 as n_name] +Query Stage #0 (1 -> 1): +SortExec: expr=[revenue@1 DESC], preserve_partitioning=[false] + ProjectionExec: expr=[n_name@0 as n_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue] + AggregateExec: mode=Single, gby=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "n_regionkey", index: 3 }, Column { name: "r_regionkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "n_regionkey", index: 3 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4)) - -Query Stage #11 (4 -> 4): -ShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "n_name", index: 0 }], 4)) - SortExec: expr=[revenue@1 DESC] - ProjectionExec: expr=[n_name@0 as n_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue] - AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "n_name", index: 0 }], 4)) - -Query Stage #12 (4 -> 1): -SortPreservingMergeExec: [revenue@1 DESC] - ShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "n_name", index: 0 }], 4)) + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, n_name@3] + ProjectionExec: expr=[r_regionkey@0 as r_regionkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: r_name@1 = AFRICA + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/region.parquet]]}, 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)] + ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, n_name@0 as n_name, n_regionkey@1 as n_regionkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_name@1, n_regionkey@2, l_extendedprice@3, l_discount@4] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, projection=[n_nationkey, n_name, n_regionkey] + ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@0 as s_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1), (s_nationkey@1, c_nationkey@0)], projection=[s_nationkey@1, l_extendedprice@4, l_discount@5] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/customer.parquet:0..13211178]]}, projection=[c_custkey, c_nationkey] + ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_orderkey, o_custkey, o_orderdate], predicate=o_orderdate@4 >= 1994-01-01 AND o_orderdate@4 < 1995-01-01, pruning_predicate=CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_max@0 >= 1994-01-01 END AND CASE WHEN o_orderdate_null_count@1 = o_orderdate_row_count@2 THEN false ELSE o_orderdate_min@3 < 1995-01-01 END, required_guarantees=[] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount] diff --git a/testdata/expected-plans/q6.txt b/testdata/expected-plans/q6.txt index 0f41810..d505b02 100644 --- a/testdata/expected-plans/q6.txt +++ b/testdata/expected-plans/q6.txt @@ -1,38 +1,30 @@ DataFusion Logical Plan ======================= -Projection: SUM(lineitem.l_extendedprice * lineitem.l_discount) AS revenue - Aggregate: groupBy=[[]], aggr=[[SUM(lineitem.l_extendedprice * lineitem.l_discount)]] +Projection: sum(lineitem.l_extendedprice * lineitem.l_discount) AS revenue + Aggregate: groupBy=[[]], aggr=[[sum(lineitem.l_extendedprice * lineitem.l_discount)]] Projection: lineitem.l_extendedprice, lineitem.l_discount - Filter: lineitem.l_shipdate >= Date32("8766") AND lineitem.l_shipdate < Date32("9131") AND lineitem.l_discount >= Decimal128(Some(3),11,2) AND lineitem.l_discount <= Decimal128(Some(5),11,2) AND lineitem.l_quantity < Decimal128(Some(2400),11,2) - TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("8766"), lineitem.l_shipdate < Date32("9131"), lineitem.l_discount >= Decimal128(Some(3),11,2), lineitem.l_discount <= Decimal128(Some(5),11,2), lineitem.l_quantity < Decimal128(Some(2400),11,2)] + Filter: lineitem.l_shipdate >= Date32("1994-01-01") AND lineitem.l_shipdate < Date32("1995-01-01") AND lineitem.l_discount >= Decimal128(Some(3),11,2) AND lineitem.l_discount <= Decimal128(Some(5),11,2) AND lineitem.l_quantity < Decimal128(Some(2400),11,2) + TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("1994-01-01"), lineitem.l_shipdate < Date32("1995-01-01"), lineitem.l_discount >= Decimal128(Some(3),11,2), lineitem.l_discount <= Decimal128(Some(5),11,2), lineitem.l_quantity < Decimal128(Some(2400),11,2)] DataFusion Physical Plan ======================== -ProjectionExec: expr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue] - AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] - CoalescePartitionsExec - AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] - ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_shipdate@3 >= 8766 AND l_shipdate@3 < 9131 AND l_discount@2 >= Some(3),11,2 AND l_discount@2 <= Some(5),11,2 AND l_quantity@0 < Some(2400),11,2 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate@10 >= 8766 AND l_shipdate@10 < 9131 AND l_discount@6 >= Some(3),11,2 AND l_discount@6 <= Some(5),11,2 AND l_quantity@4 < Some(2400),11,2, pruning_predicate=l_shipdate_max@0 >= 8766 AND l_shipdate_min@1 < 9131 AND l_discount_max@2 >= Some(3),11,2 AND l_discount_min@3 <= Some(5),11,2 AND l_quantity_min@4 < Some(2400),11,2, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate] +ProjectionExec: expr=[sum(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue] + AggregateExec: mode=Single, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] + ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(3),11,2 AND l_discount@2 <= Some(5),11,2 AND l_quantity@0 < Some(2400),11,2 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], predicate=l_shipdate@10 >= 1994-01-01 AND l_shipdate@10 < 1995-01-01 AND l_discount@6 >= Some(3),11,2 AND l_discount@6 <= Some(5),11,2 AND l_quantity@4 < Some(2400),11,2, pruning_predicate=CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_max@0 >= 1994-01-01 END AND CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_min@3 < 1995-01-01 END AND CASE WHEN l_discount_null_count@5 = l_discount_row_count@6 THEN false ELSE l_discount_max@4 >= Some(3),11,2 END AND CASE WHEN l_discount_null_count@5 = l_discount_row_count@6 THEN false ELSE l_discount_min@7 <= Some(5),11,2 END AND CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_min@8 < Some(2400),11,2 END, required_guarantees=[] RaySQL Plan =========== -Query Stage #0 (4 -> 1): -ShuffleWriterExec(stage_id=0, output_partitioning=UnknownPartitioning(4)) - AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] +Query Stage #0 (1 -> 1): +ProjectionExec: expr=[sum(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue] + AggregateExec: mode=Single, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_shipdate@3 >= 8766 AND l_shipdate@3 < 9131 AND l_discount@2 >= Some(3),11,2 AND l_discount@2 <= Some(5),11,2 AND l_quantity@0 < Some(2400),11,2 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate@10 >= 8766 AND l_shipdate@10 < 9131 AND l_discount@6 >= Some(3),11,2 AND l_discount@6 <= Some(5),11,2 AND l_quantity@4 < Some(2400),11,2, pruning_predicate=l_shipdate_max@0 >= 8766 AND l_shipdate_min@1 < 9131 AND l_discount_max@2 >= Some(3),11,2 AND l_discount_min@3 <= Some(5),11,2 AND l_quantity_min@4 < Some(2400),11,2, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate] - -Query Stage #1 (1 -> 1): -ProjectionExec: expr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue] - AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] - CoalescePartitionsExec - ShuffleReaderExec(stage_id=0, input_partitioning=UnknownPartitioning(4)) + FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(3),11,2 AND l_discount@2 <= Some(5),11,2 AND l_quantity@0 < Some(2400),11,2 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], predicate=l_shipdate@10 >= 1994-01-01 AND l_shipdate@10 < 1995-01-01 AND l_discount@6 >= Some(3),11,2 AND l_discount@6 <= Some(5),11,2 AND l_quantity@4 < Some(2400),11,2, pruning_predicate=CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_max@0 >= 1994-01-01 END AND CASE WHEN l_shipdate_null_count@1 = l_shipdate_row_count@2 THEN false ELSE l_shipdate_min@3 < 1995-01-01 END AND CASE WHEN l_discount_null_count@5 = l_discount_row_count@6 THEN false ELSE l_discount_max@4 >= Some(3),11,2 END AND CASE WHEN l_discount_null_count@5 = l_discount_row_count@6 THEN false ELSE l_discount_min@7 <= Some(5),11,2 END AND CASE WHEN l_quantity_null_count@9 = l_quantity_row_count@10 THEN false ELSE l_quantity_min@8 < Some(2400),11,2 END, required_guarantees=[] diff --git a/testdata/expected-plans/q7.txt b/testdata/expected-plans/q7.txt index 5b778e4..74d297b 100644 --- a/testdata/expected-plans/q7.txt +++ b/testdata/expected-plans/q7.txt @@ -2,192 +2,93 @@ 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)]] + 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, datepart(Utf8("YEAR"), lineitem.l_shipdate) AS l_year, CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4)) AS volume - 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, n1.n_name, n2.n_name - Inner Join: customer.c_nationkey = n2.n_nationkey - Projection: lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, customer.c_nationkey, n1.n_name - Inner Join: supplier.s_nationkey = n1.n_nationkey - Projection: supplier.s_nationkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, customer.c_nationkey - Inner Join: orders.o_custkey = customer.c_custkey - Projection: supplier.s_nationkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, orders.o_custkey - Inner Join: lineitem.l_orderkey = orders.o_orderkey - Projection: supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate - Inner Join: supplier.s_suppkey = lineitem.l_suppkey - TableScan: supplier projection=[s_suppkey, s_nationkey] - Filter: lineitem.l_shipdate >= Date32("9131") AND lineitem.l_shipdate <= Date32("9861") - TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9131"), lineitem.l_shipdate <= Date32("9861")] - 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")] + 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] - 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)] +SortExec: expr=[supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST], preserve_partitioning=[false] + 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=Single, 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 - RepartitionExec: partitioning=Hash([Column { name: "supp_nation", index: 0 }, Column { name: "cust_nation", index: 1 }, Column { name: "l_year", index: 2 }], 4), input_partitions=4 - 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@3 as supp_nation, n_name@4 as cust_nation, datepart(YEAR, l_shipdate@2) as l_year, CAST(l_extendedprice@0 AS Decimal128(35, 4)) * CAST(Some(100),23,2 - CAST(l_discount@1 AS Decimal128(23, 2)) AS Decimal128(35, 4)) as volume] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: n_name@3 = GERMANY AND n_name@4 = IRAQ OR n_name@3 = IRAQ AND n_name@4 = GERMANY - ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, l_shipdate@2 as l_shipdate, n_name@4 as n_name, n_name@6 as n_name] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_nationkey", index: 3 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "c_nationkey", index: 3 }], 4), input_partitions=4 - 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@6 as n_name] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 0 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[s_nationkey@0 as s_nationkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@6 as c_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_custkey", index: 4 }, Column { name: "c_custkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_custkey", index: 4 }], 4), input_partitions=4 - ProjectionExec: expr=[s_nationkey@0 as s_nationkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, l_shipdate@4 as l_shipdate, o_custkey@6 as o_custkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_orderkey", index: 1 }, Column { name: "o_orderkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 1 }], 4), input_partitions=4 - ProjectionExec: expr=[s_nationkey@1 as s_nationkey, l_orderkey@2 as l_orderkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, l_shipdate@6 as l_shipdate] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "l_suppkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 4), input_partitions=4 - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_shipdate@4 >= 9131 AND l_shipdate@4 <= 9861 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate@10 >= 9131 AND l_shipdate@10 <= 9861, pruning_predicate=l_shipdate_max@0 >= 9131 AND l_shipdate_min@1 <= 9861, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, projection=[o_orderkey, o_custkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, projection=[c_custkey, c_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4 - RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: n_name@1 = GERMANY OR n_name@1 = IRAQ - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, predicate=n_name@1 = GERMANY OR n_name@1 = IRAQ, pruning_predicate=n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1 OR n_name_min@0 <= IRAQ AND IRAQ <= n_name_max@1, projection=[n_nationkey, n_name] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4 - RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 + HashJoinExec: mode=CollectLeft, 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 + FilterExec: n_name@1 = IRAQ OR n_name@1 = GERMANY + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, 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)] + 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=CollectLeft, 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 + FilterExec: n_name@1 = GERMANY OR n_name@1 = IRAQ + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, 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 (IRAQ, GERMANY)] + 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=CollectLeft, 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] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/customer.parquet:0..13211178]]}, projection=[c_custkey, c_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, 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 + HashJoinExec: mode=CollectLeft, 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] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_nationkey] CoalesceBatchesExec: target_batch_size=8192 - FilterExec: n_name@1 = IRAQ OR n_name@1 = GERMANY - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, predicate=n_name@1 = IRAQ OR n_name@1 = GERMANY, pruning_predicate=n_name_min@0 <= IRAQ AND IRAQ <= n_name_max@1 OR n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, projection=[n_nationkey, n_name] + FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, 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=[] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_orderkey, o_custkey] RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] - -Query Stage #1 (4 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: l_shipdate@4 >= 9131 AND l_shipdate@4 <= 9861 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate@10 >= 9131 AND l_shipdate@10 <= 9861, pruning_predicate=l_shipdate_max@0 >= 9131 AND l_shipdate_min@1 <= 9861, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate] - -Query Stage #2 (4 -> 4): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "l_orderkey", index: 1 }], 4)) - ProjectionExec: expr=[s_nationkey@1 as s_nationkey, l_orderkey@2 as l_orderkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, l_shipdate@6 as l_shipdate] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "l_suppkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 4)) - -Query Stage #3 (4 -> 4): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, projection=[o_orderkey, o_custkey] - -Query Stage #4 (4 -> 4): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "o_custkey", index: 4 }], 4)) - ProjectionExec: expr=[s_nationkey@0 as s_nationkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, l_shipdate@4 as l_shipdate, o_custkey@6 as o_custkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_orderkey", index: 1 }, Column { name: "o_orderkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "l_orderkey", index: 1 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) - -Query Stage #5 (4 -> 4): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, projection=[c_custkey, c_nationkey] - -Query Stage #6 (4 -> 4): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "s_nationkey", index: 0 }], 4)) - ProjectionExec: expr=[s_nationkey@0 as s_nationkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@6 as c_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_custkey", index: 4 }, Column { name: "c_custkey", index: 0 })] +Query Stage #0 (1 -> 1): +SortExec: expr=[supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST], preserve_partitioning=[false] + 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=Single, 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 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "o_custkey", index: 4 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) - -Query Stage #7 (1 -> 4): -ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: n_name@1 = GERMANY OR n_name@1 = IRAQ - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, predicate=n_name@1 = GERMANY OR n_name@1 = IRAQ, pruning_predicate=n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1 OR n_name_min@0 <= IRAQ AND IRAQ <= n_name_max@1, projection=[n_nationkey, n_name] - -Query Stage #8 (4 -> 4): -ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "c_nationkey", index: 3 }], 4)) - 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@6 as n_name] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 0 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "s_nationkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - -Query Stage #9 (1 -> 4): -ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: n_name@1 = IRAQ OR n_name@1 = GERMANY - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, predicate=n_name@1 = IRAQ OR n_name@1 = GERMANY, pruning_predicate=n_name_min@0 <= IRAQ AND IRAQ <= n_name_max@1 OR n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, projection=[n_nationkey, n_name] - -Query Stage #10 (4 -> 4): -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 }], 4)) - 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@3 as supp_nation, n_name@4 as cust_nation, datepart(YEAR, l_shipdate@2) as l_year, CAST(l_extendedprice@0 AS Decimal128(35, 4)) * CAST(Some(100),23,2 - CAST(l_discount@1 AS Decimal128(23, 2)) AS Decimal128(35, 4)) as volume] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: n_name@3 = GERMANY AND n_name@4 = IRAQ OR n_name@3 = IRAQ AND n_name@4 = GERMANY - ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, l_shipdate@2 as l_shipdate, n_name@4 as n_name, n_name@6 as n_name] + HashJoinExec: mode=CollectLeft, 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 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_nationkey", index: 3 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "c_nationkey", index: 3 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - -Query Stage #11 (4 -> 4): -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 }], 4)) - SortExec: expr=[supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST] - 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 }], 4)) - -Query Stage #12 (4 -> 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 }], 4)) + FilterExec: n_name@1 = IRAQ OR n_name@1 = GERMANY + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, 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)] + 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=CollectLeft, 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 + FilterExec: n_name@1 = GERMANY OR n_name@1 = IRAQ + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, 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 (IRAQ, GERMANY)] + 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=CollectLeft, 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] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/customer.parquet:0..13211178]]}, projection=[c_custkey, c_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, 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 + HashJoinExec: mode=CollectLeft, 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] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, 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=[] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_orderkey, o_custkey] diff --git a/testdata/expected-plans/q8.txt b/testdata/expected-plans/q8.txt index c315805..6ab620c 100644 --- a/testdata/expected-plans/q8.txt +++ b/testdata/expected-plans/q8.txt @@ -2,10 +2,10 @@ DataFusion Logical Plan ======================= Sort: all_nations.o_year ASC NULLS LAST - Projection: all_nations.o_year, SUM(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END) / SUM(all_nations.volume) AS mkt_share - Aggregate: groupBy=[[all_nations.o_year]], aggr=[[SUM(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Decimal128(Some(0),35,4) END) AS SUM(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)]] + Projection: all_nations.o_year, sum(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END) / sum(all_nations.volume) AS mkt_share + Aggregate: groupBy=[[all_nations.o_year]], aggr=[[sum(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Decimal128(Some(0),35,4) END) AS sum(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)]] SubqueryAlias: all_nations - Projection: datepart(Utf8("YEAR"), orders.o_orderdate) AS o_year, CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4)) AS volume, n2.n_name AS nation + Projection: date_part(Utf8("YEAR"), orders.o_orderdate) AS o_year, lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS volume, n2.n_name AS nation Inner Join: n1.n_regionkey = region.r_regionkey Projection: lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, n1.n_regionkey, n2.n_name Inner Join: supplier.s_nationkey = n2.n_nationkey @@ -24,8 +24,8 @@ Sort: all_nations.o_year ASC NULLS LAST TableScan: part projection=[p_partkey, p_type], partial_filters=[part.p_type = Utf8("LARGE PLATED STEEL")] TableScan: lineitem projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount] TableScan: supplier projection=[s_suppkey, s_nationkey] - Filter: orders.o_orderdate >= Date32("9131") AND orders.o_orderdate <= Date32("9861") - TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("9131"), orders.o_orderdate <= Date32("9861")] + Filter: orders.o_orderdate >= Date32("1995-01-01") AND orders.o_orderdate <= Date32("1996-12-31") + TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("1995-01-01"), orders.o_orderdate <= Date32("1996-12-31")] TableScan: customer projection=[c_custkey, c_nationkey] SubqueryAlias: n1 TableScan: nation projection=[n_nationkey, n_regionkey] @@ -38,203 +38,87 @@ Sort: all_nations.o_year ASC NULLS LAST DataFusion Physical Plan ======================== -SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] - SortExec: expr=[o_year@0 ASC NULLS LAST] - ProjectionExec: expr=[o_year@0 as o_year, SUM(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END)@1 / SUM(all_nations.volume)@2 as mkt_share] - AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] +SortExec: expr=[o_year@0 ASC NULLS LAST], preserve_partitioning=[false] + ProjectionExec: expr=[o_year@0 as o_year, sum(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END)@1 / sum(all_nations.volume)@2 as mkt_share] + AggregateExec: mode=Single, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] + ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_year", index: 0 }], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] - ProjectionExec: expr=[datepart(YEAR, o_orderdate@2) as o_year, CAST(l_extendedprice@0 AS Decimal128(35, 4)) * CAST(Some(100),23,2 - CAST(l_discount@1 AS Decimal128(23, 2)) AS Decimal128(35, 4)) as volume, n_name@4 as nation] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "n_regionkey", index: 3 }, Column { name: "r_regionkey", index: 0 })] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, o_orderdate@3, n_name@5] + ProjectionExec: expr=[r_regionkey@0 as r_regionkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: r_name@1 = MIDDLE EAST + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/region.parquet]]}, 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)] + ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, o_orderdate@3 as o_orderdate, n_regionkey@4 as n_regionkey, n_name@0 as n_name] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_name@1, l_extendedprice@2, l_discount@3, o_orderdate@5, n_regionkey@6] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, projection=[n_nationkey, n_name] + ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@3 as s_nationkey, o_orderdate@4 as o_orderdate, n_regionkey@0 as n_regionkey] CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "n_regionkey", index: 3 }], 4), input_partitions=4 - ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, o_orderdate@3 as o_orderdate, n_regionkey@4 as n_regionkey, n_name@6 as n_name] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@4)], projection=[n_regionkey@1, l_extendedprice@2, l_discount@3, s_nationkey@4, o_orderdate@5] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, projection=[n_nationkey, n_regionkey] + ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@3 as s_nationkey, o_orderdate@4 as o_orderdate, c_nationkey@0 as c_nationkey] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 2 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 4), input_partitions=4 - ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, s_nationkey@2 as s_nationkey, o_orderdate@3 as o_orderdate, n_regionkey@6 as n_regionkey] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@3)], projection=[c_nationkey@1, l_extendedprice@2, l_discount@3, s_nationkey@4, o_orderdate@6] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/customer.parquet:0..13211178]]}, projection=[c_custkey, c_nationkey] + ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@4 as s_nationkey, o_custkey@0 as o_custkey, o_orderdate@1 as o_orderdate] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_custkey@1, o_orderdate@2, l_extendedprice@4, l_discount@5, s_nationkey@6] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_nationkey", index: 4 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "c_nationkey", index: 4 }], 4), input_partitions=4 - ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, s_nationkey@2 as s_nationkey, o_orderdate@4 as o_orderdate, c_nationkey@6 as c_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_custkey", index: 3 }, Column { name: "c_custkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_custkey", index: 3 }], 4), input_partitions=4 - ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@3 as s_nationkey, o_custkey@5 as o_custkey, o_orderdate@6 as o_orderdate] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@5 as s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_suppkey", index: 1 }, Column { name: "s_suppkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 4), input_partitions=4 - ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_suppkey@3 as l_suppkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 0 }, Column { name: "l_partkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[p_partkey@0 as p_partkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: p_type@1 = LARGE PLATED STEEL - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_type@4 = LARGE PLATED STEEL, pruning_predicate=p_type_min@0 <= LARGE PLATED STEEL AND LARGE PLATED STEEL <= p_type_max@1, projection=[p_partkey, p_type] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_partkey", index: 1 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4), input_partitions=4 - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: o_orderdate@2 >= 9131 AND o_orderdate@2 <= 9861 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderdate@4 >= 9131 AND o_orderdate@4 <= 9861, pruning_predicate=o_orderdate_max@0 >= 9131 AND o_orderdate_min@1 <= 9861, projection=[o_orderkey, o_custkey, o_orderdate] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, projection=[c_custkey, c_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=1 - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_regionkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=1 - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_name] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4), input_partitions=4 - RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - ProjectionExec: expr=[r_regionkey@0 as r_regionkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: r_name@1 = MIDDLE EAST - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/region.parquet/part-0.parquet]]}, predicate=r_name@1 = MIDDLE EAST, pruning_predicate=r_name_min@0 <= MIDDLE EAST AND MIDDLE EAST <= r_name_max@1, projection=[r_regionkey, r_name] + FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, 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=[] + ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@0 as s_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] + ProjectionExec: expr=[p_partkey@0 as p_partkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: p_type@1 = LARGE PLATED STEEL + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/part.parquet]]}, projection=[p_partkey, p_type], predicate=p_type@4 = LARGE PLATED STEEL, pruning_predicate=CASE WHEN p_type_null_count@2 = p_type_row_count@3 THEN false ELSE p_type_min@0 <= LARGE PLATED STEEL AND LARGE PLATED STEEL <= p_type_max@1 END, required_guarantees=[p_type in (LARGE PLATED STEEL)] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount] RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) - ProjectionExec: expr=[p_partkey@0 as p_partkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: p_type@1 = LARGE PLATED STEEL - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_type@4 = LARGE PLATED STEEL, pruning_predicate=p_type_min@0 <= LARGE PLATED STEEL AND LARGE PLATED STEEL <= p_type_max@1, projection=[p_partkey, p_type] - -Query Stage #1 (4 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_partkey", index: 1 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount] - -Query Stage #2 (4 -> 4): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 4)) - ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_suppkey@3 as l_suppkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 0 }, Column { name: "l_partkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_partkey", index: 1 }], 4)) - -Query Stage #3 (4 -> 4): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] - -Query Stage #4 (4 -> 4): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@5 as s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_suppkey", index: 1 }, Column { name: "s_suppkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - -Query Stage #5 (4 -> 4): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: o_orderdate@2 >= 9131 AND o_orderdate@2 <= 9861 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderdate@4 >= 9131 AND o_orderdate@4 <= 9861, pruning_predicate=o_orderdate_max@0 >= 9131 AND o_orderdate_min@1 <= 9861, projection=[o_orderkey, o_custkey, o_orderdate] - -Query Stage #6 (4 -> 4): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "o_custkey", index: 3 }], 4)) - ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@3 as s_nationkey, o_custkey@5 as o_custkey, o_orderdate@6 as o_orderdate] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) - -Query Stage #7 (4 -> 4): -ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, projection=[c_custkey, c_nationkey] - -Query Stage #8 (4 -> 4): -ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "c_nationkey", index: 4 }], 4)) - ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, s_nationkey@2 as s_nationkey, o_orderdate@4 as o_orderdate, c_nationkey@6 as c_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_custkey", index: 3 }, Column { name: "c_custkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "o_custkey", index: 3 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) - -Query Stage #9 (1 -> 4): -ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_regionkey] - -Query Stage #10 (4 -> 4): -ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 4)) - ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, s_nationkey@2 as s_nationkey, o_orderdate@3 as o_orderdate, n_regionkey@6 as n_regionkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_nationkey", index: 4 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "c_nationkey", index: 4 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - -Query Stage #11 (1 -> 4): -ShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_name] - -Query Stage #12 (4 -> 4): -ShuffleWriterExec(stage_id=12, output_partitioning=Hash([Column { name: "n_regionkey", index: 3 }], 4)) - ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, o_orderdate@3 as o_orderdate, n_regionkey@4 as n_regionkey, n_name@6 as n_name] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 2 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - -Query Stage #13 (1 -> 4): -ShuffleWriterExec(stage_id=13, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4)) - ProjectionExec: expr=[r_regionkey@0 as r_regionkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: r_name@1 = MIDDLE EAST - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/region.parquet/part-0.parquet]]}, predicate=r_name@1 = MIDDLE EAST, pruning_predicate=r_name_min@0 <= MIDDLE EAST AND MIDDLE EAST <= r_name_max@1, projection=[r_regionkey, r_name] - -Query Stage #14 (4 -> 4): -ShuffleWriterExec(stage_id=14, output_partitioning=Hash([Column { name: "o_year", index: 0 }], 4)) - AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] - ProjectionExec: expr=[datepart(YEAR, o_orderdate@2) as o_year, CAST(l_extendedprice@0 AS Decimal128(35, 4)) * CAST(Some(100),23,2 - CAST(l_discount@1 AS Decimal128(23, 2)) AS Decimal128(35, 4)) as volume, n_name@4 as nation] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "n_regionkey", index: 3 }, Column { name: "r_regionkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=12, input_partitioning=Hash([Column { name: "n_regionkey", index: 3 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=13, input_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4)) - -Query Stage #15 (4 -> 4): -ShuffleWriterExec(stage_id=15, output_partitioning=Hash([Column { name: "o_year", index: 0 }], 4)) - SortExec: expr=[o_year@0 ASC NULLS LAST] - ProjectionExec: expr=[o_year@0 as o_year, SUM(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END)@1 / SUM(all_nations.volume)@2 as mkt_share] - AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] +Query Stage #0 (1 -> 1): +SortExec: expr=[o_year@0 ASC NULLS LAST], preserve_partitioning=[false] + ProjectionExec: expr=[o_year@0 as o_year, sum(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END)@1 / sum(all_nations.volume)@2 as mkt_share] + AggregateExec: mode=Single, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] + ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=14, input_partitioning=Hash([Column { name: "o_year", index: 0 }], 4)) - -Query Stage #16 (4 -> 1): -SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] - ShuffleReaderExec(stage_id=15, input_partitioning=Hash([Column { name: "o_year", index: 0 }], 4)) + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, o_orderdate@3, n_name@5] + ProjectionExec: expr=[r_regionkey@0 as r_regionkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: r_name@1 = MIDDLE EAST + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/region.parquet]]}, 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)] + ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, o_orderdate@3 as o_orderdate, n_regionkey@4 as n_regionkey, n_name@0 as n_name] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_name@1, l_extendedprice@2, l_discount@3, o_orderdate@5, n_regionkey@6] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, projection=[n_nationkey, n_name] + ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@3 as s_nationkey, o_orderdate@4 as o_orderdate, n_regionkey@0 as n_regionkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@4)], projection=[n_regionkey@1, l_extendedprice@2, l_discount@3, s_nationkey@4, o_orderdate@5] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, projection=[n_nationkey, n_regionkey] + ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@3 as s_nationkey, o_orderdate@4 as o_orderdate, c_nationkey@0 as c_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@3)], projection=[c_nationkey@1, l_extendedprice@2, l_discount@3, s_nationkey@4, o_orderdate@6] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/customer.parquet:0..13211178]]}, projection=[c_custkey, c_nationkey] + ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@4 as s_nationkey, o_custkey@0 as o_custkey, o_orderdate@1 as o_orderdate] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_custkey@1, o_orderdate@2, l_extendedprice@4, l_discount@5, s_nationkey@6] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, 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=[] + ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@0 as s_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] + ProjectionExec: expr=[p_partkey@0 as p_partkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: p_type@1 = LARGE PLATED STEEL + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/part.parquet]]}, projection=[p_partkey, p_type], predicate=p_type@4 = LARGE PLATED STEEL, pruning_predicate=CASE WHEN p_type_null_count@2 = p_type_row_count@3 THEN false ELSE p_type_min@0 <= LARGE PLATED STEEL AND LARGE PLATED STEEL <= p_type_max@1 END, required_guarantees=[p_type in (LARGE PLATED STEEL)] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount] diff --git a/testdata/expected-plans/q9.txt b/testdata/expected-plans/q9.txt index 5afd3dc..f305294 100644 --- a/testdata/expected-plans/q9.txt +++ b/testdata/expected-plans/q9.txt @@ -2,10 +2,10 @@ DataFusion Logical Plan ======================= Sort: profit.nation ASC NULLS LAST, profit.o_year DESC NULLS FIRST - Projection: profit.nation, profit.o_year, SUM(profit.amount) AS sum_profit - Aggregate: groupBy=[[profit.nation, profit.o_year]], aggr=[[SUM(profit.amount)]] + Projection: profit.nation, profit.o_year, sum(profit.amount) AS sum_profit + Aggregate: groupBy=[[profit.nation, profit.o_year]], aggr=[[sum(profit.amount)]] SubqueryAlias: profit - Projection: nation.n_name AS nation, datepart(Utf8("YEAR"), orders.o_orderdate) AS o_year, CAST(CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4)) AS Decimal128(36, 4)) - CAST(partsupp.ps_supplycost * lineitem.l_quantity AS Decimal128(36, 4)) AS amount + Projection: nation.n_name AS nation, date_part(Utf8("YEAR"), orders.o_orderdate) AS o_year, lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) - partsupp.ps_supplycost * lineitem.l_quantity AS amount Inner Join: supplier.s_nationkey = nation.n_nationkey Projection: lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, partsupp.ps_supplycost, orders.o_orderdate Inner Join: lineitem.l_orderkey = orders.o_orderkey @@ -27,148 +27,61 @@ Sort: profit.nation ASC NULLS LAST, profit.o_year DESC NULLS FIRST DataFusion Physical Plan ======================== -SortPreservingMergeExec: [nation@0 ASC NULLS LAST,o_year@1 DESC] - SortExec: expr=[nation@0 ASC NULLS LAST,o_year@1 DESC] - ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, SUM(profit.amount)@2 as sum_profit] - AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] +SortExec: expr=[nation@0 ASC NULLS LAST,o_year@1 DESC], preserve_partitioning=[false] + ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, sum(profit.amount)@2 as sum_profit] + AggregateExec: mode=Single, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] + ProjectionExec: expr=[n_name@0 as nation, date_part(YEAR, o_orderdate@5) as o_year, l_extendedprice@2 * (Some(1),20,0 - l_discount@3) - ps_supplycost@4 * l_quantity@1 as amount] CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "nation", index: 0 }, Column { name: "o_year", index: 1 }], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] - ProjectionExec: expr=[n_name@7 as nation, datepart(YEAR, o_orderdate@5) as o_year, CAST(CAST(l_extendedprice@1 AS Decimal128(35, 4)) * CAST(Some(100),23,2 - CAST(l_discount@2 AS Decimal128(23, 2)) AS Decimal128(35, 4)) AS Decimal128(36, 4)) - CAST(ps_supplycost@4 * l_quantity@0 AS Decimal128(36, 4)) as amount] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 3 }, Column { name: "n_nationkey", index: 0 })] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[n_name@1, l_quantity@2, l_extendedprice@3, l_discount@4, ps_supplycost@6, o_orderdate@7] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, projection=[n_nationkey, n_name] + ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@4 as s_nationkey, ps_supplycost@5 as ps_supplycost, o_orderdate@0 as o_orderdate] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@7] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_orderkey, o_orderdate] + ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_quantity@2 as l_quantity, l_extendedprice@3 as l_extendedprice, l_discount@4 as l_discount, s_nationkey@5 as s_nationkey, ps_supplycost@0 as ps_supplycost] CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 4), input_partitions=4 - ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@4 as s_nationkey, ps_supplycost@5 as ps_supplycost, o_orderdate@7 as o_orderdate] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ps_suppkey@1, l_suppkey@2), (ps_partkey@0, l_partkey@1)], projection=[ps_supplycost@2, l_orderkey@3, l_quantity@6, l_extendedprice@7, l_discount@8, s_nationkey@9] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/partsupp.parquet:0..41091907]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost] + ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_partkey@2 as l_partkey, l_suppkey@3 as l_suppkey, l_quantity@4 as l_quantity, l_extendedprice@5 as l_extendedprice, l_discount@6 as l_discount, s_nationkey@0 as s_nationkey] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderkey", index: 0 })] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@2)], projection=[s_nationkey@1, l_orderkey@2, l_partkey@3, l_suppkey@4, l_quantity@5, l_extendedprice@6, l_discount@7] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_nationkey] CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_quantity@3 as l_quantity, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, s_nationkey@6 as s_nationkey, ps_supplycost@9 as ps_supplycost] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_partkey@2, l_suppkey@3, l_quantity@4, l_extendedprice@5, l_discount@6] + ProjectionExec: expr=[p_partkey@0 as p_partkey] CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_suppkey", index: 2 }, Column { name: "ps_suppkey", index: 1 }), (Column { name: "l_partkey", index: 1 }, Column { name: "ps_partkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_suppkey", index: 2 }, Column { name: "l_partkey", index: 1 }], 4), input_partitions=4 - ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_partkey@1 as l_partkey, l_suppkey@2 as l_suppkey, l_quantity@3 as l_quantity, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, s_nationkey@7 as s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_suppkey", index: 2 }, Column { name: "s_suppkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_suppkey", index: 2 }], 4), input_partitions=4 - ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_partkey@2 as l_partkey, l_suppkey@3 as l_suppkey, l_quantity@4 as l_quantity, l_extendedprice@5 as l_extendedprice, l_discount@6 as l_discount] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 0 }, Column { name: "l_partkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4), input_partitions=4 - ProjectionExec: expr=[p_partkey@0 as p_partkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: p_name@1 LIKE %moccasin% - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_name@1 LIKE %moccasin%, projection=[p_partkey, p_name] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "l_partkey", index: 1 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "ps_suppkey", index: 1 }, Column { name: "ps_partkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4), input_partitions=4 - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, projection=[o_orderkey, o_orderdate] - CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=1 - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_name] + FilterExec: p_name@1 LIKE %moccasin% + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/part.parquet]]}, projection=[p_partkey, p_name], predicate=p_name@1 LIKE %moccasin% + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount] RaySQL Plan =========== -Query Stage #0 (4 -> 4): -ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) - ProjectionExec: expr=[p_partkey@0 as p_partkey] - CoalesceBatchesExec: target_batch_size=8192 - FilterExec: p_name@1 LIKE %moccasin% - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_name@1 LIKE %moccasin%, projection=[p_partkey, p_name] - -Query Stage #1 (4 -> 4): -ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_partkey", index: 1 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount] - -Query Stage #2 (4 -> 4): -ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "l_suppkey", index: 2 }], 4)) - ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_partkey@2 as l_partkey, l_suppkey@3 as l_suppkey, l_quantity@4 as l_quantity, l_extendedprice@5 as l_extendedprice, l_discount@6 as l_discount] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 0 }, Column { name: "l_partkey", index: 1 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_partkey", index: 1 }], 4)) - -Query Stage #3 (4 -> 4): -ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] - -Query Stage #4 (4 -> 4): -ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_suppkey", index: 2 }, Column { name: "l_partkey", index: 1 }], 4)) - ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_partkey@1 as l_partkey, l_suppkey@2 as l_suppkey, l_quantity@3 as l_quantity, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, s_nationkey@7 as s_nationkey] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_suppkey", index: 2 }, Column { name: "s_suppkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "l_suppkey", index: 2 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) - -Query Stage #5 (4 -> 4): -ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }, Column { name: "ps_partkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost] - -Query Stage #6 (4 -> 4): -ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) - ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_quantity@3 as l_quantity, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, s_nationkey@6 as s_nationkey, ps_supplycost@9 as ps_supplycost] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_suppkey", index: 2 }, Column { name: "ps_suppkey", index: 1 }), (Column { name: "l_partkey", index: 1 }, Column { name: "ps_partkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_suppkey", index: 2 }, Column { name: "l_partkey", index: 1 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }, Column { name: "ps_partkey", index: 0 }], 4)) - -Query Stage #7 (4 -> 4): -ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, projection=[o_orderkey, o_orderdate] - -Query Stage #8 (4 -> 4): -ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 4)) - ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@4 as s_nationkey, ps_supplycost@5 as ps_supplycost, o_orderdate@7 as o_orderdate] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) +Query Stage #0 (1 -> 1): +SortExec: expr=[nation@0 ASC NULLS LAST,o_year@1 DESC], preserve_partitioning=[false] + ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, sum(profit.amount)@2 as sum_profit] + AggregateExec: mode=Single, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] + ProjectionExec: expr=[n_name@0 as nation, date_part(YEAR, o_orderdate@5) as o_year, l_extendedprice@2 * (Some(1),20,0 - l_discount@3) - ps_supplycost@4 * l_quantity@1 as amount] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) - -Query Stage #9 (1 -> 4): -ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_name] - -Query Stage #10 (4 -> 4): -ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "nation", index: 0 }, Column { name: "o_year", index: 1 }], 4)) - AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] - ProjectionExec: expr=[n_name@7 as nation, datepart(YEAR, o_orderdate@5) as o_year, CAST(CAST(l_extendedprice@1 AS Decimal128(35, 4)) * CAST(Some(100),23,2 - CAST(l_discount@2 AS Decimal128(23, 2)) AS Decimal128(35, 4)) AS Decimal128(36, 4)) - CAST(ps_supplycost@4 * l_quantity@0 AS Decimal128(36, 4)) as amount] - CoalesceBatchesExec: target_batch_size=8192 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 3 }, Column { name: "n_nationkey", index: 0 })] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 4)) - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) - -Query Stage #11 (4 -> 4): -ShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "nation", index: 0 }, Column { name: "o_year", index: 1 }], 4)) - SortExec: expr=[nation@0 ASC NULLS LAST,o_year@1 DESC] - ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, SUM(profit.amount)@2 as sum_profit] - AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] - CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "nation", index: 0 }, Column { name: "o_year", index: 1 }], 4)) - -Query Stage #12 (4 -> 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 }], 4)) + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[n_name@1, l_quantity@2, l_extendedprice@3, l_discount@4, ps_supplycost@6, o_orderdate@7] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/nation.parquet]]}, projection=[n_nationkey, n_name] + ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@4 as s_nationkey, ps_supplycost@5 as ps_supplycost, o_orderdate@0 as o_orderdate] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@7] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/orders.parquet:0..54530383]]}, projection=[o_orderkey, o_orderdate] + ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_quantity@2 as l_quantity, l_extendedprice@3 as l_extendedprice, l_discount@4 as l_discount, s_nationkey@5 as s_nationkey, ps_supplycost@0 as ps_supplycost] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ps_suppkey@1, l_suppkey@2), (ps_partkey@0, l_partkey@1)], projection=[ps_supplycost@2, l_orderkey@3, l_quantity@6, l_extendedprice@7, l_discount@8, s_nationkey@9] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/partsupp.parquet:0..41091907]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost] + ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_partkey@2 as l_partkey, l_suppkey@3 as l_suppkey, l_quantity@4 as l_quantity, l_extendedprice@5 as l_extendedprice, l_discount@6 as l_discount, s_nationkey@0 as s_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@2)], projection=[s_nationkey@1, l_orderkey@2, l_partkey@3, l_suppkey@4, l_quantity@5, l_extendedprice@6, l_discount@7] + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/supplier.parquet]]}, projection=[s_suppkey, s_nationkey] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_partkey@2, l_suppkey@3, l_quantity@4, l_extendedprice@5, l_discount@6] + ProjectionExec: expr=[p_partkey@0 as p_partkey] + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: p_name@1 LIKE %moccasin% + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/part.parquet]]}, projection=[p_partkey, p_name], predicate=p_name@1 LIKE %moccasin% + ParquetExec: file_groups={1 group: [[home/runner/work/datafusion-ray/datafusion-ray/data/lineitem.parquet:0..208317955]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount] diff --git a/tpch/requirements.txt b/tpch/requirements.txt new file mode 100644 index 0000000..2d257db --- /dev/null +++ b/tpch/requirements.txt @@ -0,0 +1,4 @@ +# This is a bad idea, we should lock dependencies with poetry and consume this tool as an action +pyarrow +datafusion +argparse diff --git a/tpch/tpchgen.py b/tpch/tpchgen.py new file mode 100644 index 0000000..2425916 --- /dev/null +++ b/tpch/tpchgen.py @@ -0,0 +1,264 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import argparse +import concurrent.futures +from datafusion import SessionContext +import os +import pyarrow +import subprocess +import time + +table_names = [ + "customer", + "lineitem", + "nation", + "orders", + "part", + "partsupp", + "region", + "supplier", +] + +# schema definition copied from DataFusion Python tpch example +all_schemas = {} + +all_schemas["customer"] = [ + ("C_CUSTKEY", pyarrow.int64()), + ("C_NAME", pyarrow.string()), + ("C_ADDRESS", pyarrow.string()), + ("C_NATIONKEY", pyarrow.int64()), + ("C_PHONE", pyarrow.string()), + ("C_ACCTBAL", pyarrow.decimal128(11, 2)), + ("C_MKTSEGMENT", pyarrow.string()), + ("C_COMMENT", pyarrow.string()), +] + +all_schemas["lineitem"] = [ + ("L_ORDERKEY", pyarrow.int64()), + ("L_PARTKEY", pyarrow.int64()), + ("L_SUPPKEY", pyarrow.int64()), + ("L_LINENUMBER", pyarrow.int32()), + ("L_QUANTITY", pyarrow.decimal128(11, 2)), + ("L_EXTENDEDPRICE", pyarrow.decimal128(11, 2)), + ("L_DISCOUNT", pyarrow.decimal128(11, 2)), + ("L_TAX", pyarrow.decimal128(11, 2)), + ("L_RETURNFLAG", pyarrow.string()), + ("L_LINESTATUS", pyarrow.string()), + ("L_SHIPDATE", pyarrow.date32()), + ("L_COMMITDATE", pyarrow.date32()), + ("L_RECEIPTDATE", pyarrow.date32()), + ("L_SHIPINSTRUCT", pyarrow.string()), + ("L_SHIPMODE", pyarrow.string()), + ("L_COMMENT", pyarrow.string()), +] + +all_schemas["nation"] = [ + ("N_NATIONKEY", pyarrow.int64()), + ("N_NAME", pyarrow.string()), + ("N_REGIONKEY", pyarrow.int64()), + ("N_COMMENT", pyarrow.string()), +] + +all_schemas["orders"] = [ + ("O_ORDERKEY", pyarrow.int64()), + ("O_CUSTKEY", pyarrow.int64()), + ("O_ORDERSTATUS", pyarrow.string()), + ("O_TOTALPRICE", pyarrow.decimal128(11, 2)), + ("O_ORDERDATE", pyarrow.date32()), + ("O_ORDERPRIORITY", pyarrow.string()), + ("O_CLERK", pyarrow.string()), + ("O_SHIPPRIORITY", pyarrow.int32()), + ("O_COMMENT", pyarrow.string()), +] + +all_schemas["part"] = [ + ("P_PARTKEY", pyarrow.int64()), + ("P_NAME", pyarrow.string()), + ("P_MFGR", pyarrow.string()), + ("P_BRAND", pyarrow.string()), + ("P_TYPE", pyarrow.string()), + ("P_SIZE", pyarrow.int32()), + ("P_CONTAINER", pyarrow.string()), + ("P_RETAILPRICE", pyarrow.decimal128(11, 2)), + ("P_COMMENT", pyarrow.string()), +] + +all_schemas["partsupp"] = [ + ("PS_PARTKEY", pyarrow.int64()), + ("PS_SUPPKEY", pyarrow.int64()), + ("PS_AVAILQTY", pyarrow.int32()), + ("PS_SUPPLYCOST", pyarrow.decimal128(11, 2)), + ("PS_COMMENT", pyarrow.string()), +] + +all_schemas["region"] = [ + ("R_REGIONKEY", pyarrow.int64()), + ("R_NAME", pyarrow.string()), + ("R_COMMENT", pyarrow.string()), +] + +all_schemas["supplier"] = [ + ("S_SUPPKEY", pyarrow.int64()), + ("S_NAME", pyarrow.string()), + ("S_ADDRESS", pyarrow.string()), + ("S_NATIONKEY", pyarrow.int64()), + ("S_PHONE", pyarrow.string()), + ("S_ACCTBAL", pyarrow.decimal128(11, 2)), + ("S_COMMENT", pyarrow.string()), +] + + +def run(cmd: str): + print(f"Executing: {cmd}") + subprocess.run(cmd, shell=True, check=True) + + +def run_and_log_output(cmd: str, log_file: str): + print(f"Executing: {cmd}; writing output to {log_file}") + with open(log_file, "w") as file: + subprocess.run( + cmd, shell=True, check=True, stdout=file, stderr=subprocess.STDOUT + ) + + +def convert_tbl_to_parquet( + ctx: SessionContext, + table: str, + tbl_filename: str, + file_extension: str, + parquet_filename: str, +): + print(f"Converting {tbl_filename} to {parquet_filename} ...") + + # schema manipulation code copied from DataFusion Python tpch example + table_schema = [ + pyarrow.field(r[0].lower(), r[1], nullable=False) for r in all_schemas[table] + ] + + # Pre-collect the output columns so we can ignore the null field we add + # in to handle the trailing | in the file + output_cols = [r.name for r in table_schema] + + # Trailing | requires extra field for in processing + table_schema.append(pyarrow.field("some_null", pyarrow.null(), nullable=True)) + + schema = pyarrow.schema(table_schema) + + df = ctx.read_csv( + tbl_filename, + schema=schema, + has_header=False, + file_extension=file_extension, + delimiter="|", + ) + df = df.select_columns(*output_cols) + df.write_parquet(parquet_filename, compression="snappy") + + +def generate_tpch(scale_factor: int, partitions: int): + start_time = time.time() + docker_cmd = os.getenv("DOCKER_CMD", "docker") + if partitions == 1: + command = f"{docker_cmd} run -v `pwd`/data:/data -t --rm ghcr.io/scalytics/tpch-docker:main -vf -s {scale_factor} -r 1" + run_and_log_output(command, "/tmp/tpchgen.log") + else: + max_threads = os.cpu_count() + + # List of commands to run + commands = [ + ( + f"{docker_cmd} run -v `pwd`/data:/data -t --rm ghcr.io/scalytics/tpch-docker:main -vf -s {scale_factor} -C {partitions} -S {part} -r 1", + f"/tmp/tpchgen-part{part}.log", + ) + for part in range(1, partitions + 1) + ] + + # run commands in parallel + with concurrent.futures.ThreadPoolExecutor(max_workers=max_threads) as executor: + futures = [ + executor.submit(run_and_log_output, command, log_file) + for (command, log_file) in commands + ] + + # wait for all futures to complete + for future in concurrent.futures.as_completed(futures): + try: + future.result() + except Exception as e: + print(f"Command failed with exception: {e}") + + end_time = time.time() + print(f"Generated CSV data in {round(end_time - start_time, 2)} seconds") + + +def convert_tpch(partitions: int): + start_time = time.time() + ctx = SessionContext() + if partitions == 1: + # convert to parquet + for table in table_names: + convert_tbl_to_parquet( + ctx, table, f"data/{table}.tbl", "tbl", f"data/{table}.parquet" + ) + else: + for table in table_names: + run(f"mkdir -p data/{table}.parquet") + if table == "nation" or table == "region": + # nation and region are special cases and do not generate multiple files + convert_tbl_to_parquet( + ctx, + table, + f"data/{table}.tbl", + "tbl", + f"data/{table}.parquet/part1.parquet", + ) + else: + for part in range(1, partitions + 1): + convert_tbl_to_parquet( + ctx, + table, + f"data/{table}.tbl.{part}", + f"tbl.{part}", + f"data/{table}.parquet/part{part}.parquet", + ) + end_time = time.time() + print(f"Converted CSV to Parquet in {round(end_time - start_time, 2)} seconds") + + +if __name__ == "__main__": + arg_parser = argparse.ArgumentParser() + subparsers = arg_parser.add_subparsers(dest="command", help="Available commands") + + parser_generate = subparsers.add_parser("generate", help="Generate TPC-H CSV Data") + parser_generate.add_argument("--scale-factor", type=int, help="The scale factor") + parser_generate.add_argument( + "--partitions", type=int, help="The number of partitions" + ) + + parser_convert = subparsers.add_parser( + "convert", help="Convert TPC-H CSV Data to Parquet" + ) + parser_convert.add_argument( + "--partitions", type=int, help="The number of partitions" + ) + + args = arg_parser.parse_args() + if args.command == "generate": + generate_tpch(args.scale_factor, args.partitions) + elif args.command == "convert": + convert_tpch(args.partitions)