From 99ed391da3ce27e5787f9fd29502355acfb786cc Mon Sep 17 00:00:00 2001 From: Austin Liu Date: Mon, 30 Sep 2024 22:16:20 +0800 Subject: [PATCH] Donate Ray SQL to the DataFusion Project (#1) * Initial commit * Basic project structure * gitignore * Add protobuf plumbing (#2) * Implement protobuf codec * Wire up query execution (#4) * query runs end to end (#5) * re-organize python code (#6) * Implement shuffle more fully (#7) * update README (#8) * Bug fix (#9) * Support multiple shuffle partitions (#10) * More shuffle fixes (#11) * fix readme (#12) * add perf chart (#13) * Remove hard-coded temp dir (#14) * bug fix (#15) * New results (#16) * Upgrade to DataFusion 17, fix a couple of bugs, add some tests (#18) * Remove debug logging (#19) * update README (#21) * Make better use of futures (#23) * Documentation & bug fixes (#24) * Update README.md * [WIP] Use Ray object store for shuffle exchange (#28) * Fixes for Ray-based shuffle (#29) * Small fixes for Context (#30) * Make distributed execution work (#33) * Make distributed execution work * fix tips.py * fixes; incorporate changes from #32 * Upgrade to DataFusion 20 (#31) * Add support for DDL statements, such as `CREATE VIEW` (#35) * Experimenting with supporting DDL * update docs * Use PyArrow for zero-copy interaction with the Ray Object Store (#36) * Optimize Ray shuffle with zero-copy object store * remove more clones * change bytes to pyarrow.array * revert /tmp * remove empty_result_set * remove empty_result_set * Fix input partition count bug * Add Frank as author (#37) * fix hyperlink of issue 22 in docs/README.txt (#40) Co-authored-by: ivanfan * delta lake and iceberg table support (#43) * delta support * imports * Update DataFusion version to 28.0.0 (#41) * Update DataFusion version * update example * Upgrade to DataFusion 33 (#45) * Upgrade to DataFusion 33 * undo release profile change * Add basic GitHub workflow to compile code (#47) * Create rust.yml * install protobuf * fix * fix * fix * fix * fix * fix * Add ASF license header Signed-off-by: Austin Liu * Remove ASF header for generated code Signed-off-by: Austin Liu --------- Signed-off-by: Austin Liu Co-authored-by: Andy Grove Co-authored-by: Frank Luan Co-authored-by: Frank Luan Co-authored-by: Ivankings <37607467+Ivankings@users.noreply.github.com> Co-authored-by: ivanfan Co-authored-by: raviranak <126759945+raviranak@users.noreply.github.com> --- .github/workflows/rust.yml | 25 + .gitignore | 7 + Cargo.lock | 3268 +++++++++++++++++++++++++++++ Cargo.toml | 57 + LICENSE | 201 ++ NOTICE | 2 + README.md | 150 +- build.rs | 54 + docs/README.md | 279 +++ docs/sqlbench-h-per-query.png | Bin 0 -> 20772 bytes docs/sqlbench-h-total.png | Bin 0 -> 14479 bytes examples/tips.csv | 245 +++ examples/tips.parquet | Bin 0 -> 5884 bytes examples/tips.py | 39 + pyproject.toml | 31 + raysql/__init__.py | 33 + raysql/context.py | 255 +++ raysql/main.py | 124 ++ raysql/ray_utils.py | 31 + raysql/tests/test_context.py | 24 + requirements-in.txt | 11 + src/context.rs | 261 +++ src/lib.rs | 43 + src/planner.rs | 439 ++++ src/proto/datafusion.proto | 1347 ++++++++++++ src/proto/generated/mod.rs | 18 + src/proto/generated/protobuf.rs | 87 + src/proto/mod.rs | 18 + src/proto/raysql.proto | 58 + src/query_stage.rs | 121 ++ src/shuffle/codec.rs | 211 ++ src/shuffle/mod.rs | 27 + src/shuffle/ray_shuffle/mod.rs | 95 + src/shuffle/ray_shuffle/reader.rs | 188 ++ src/shuffle/ray_shuffle/writer.rs | 238 +++ src/shuffle/reader.rs | 185 ++ src/shuffle/writer.rs | 304 +++ src/utils.rs | 30 + testdata/expected-plans/q1.txt | 48 + testdata/expected-plans/q10.txt | 134 ++ testdata/expected-plans/q11.txt | 188 ++ testdata/expected-plans/q12.txt | 73 + testdata/expected-plans/q13.txt | 85 + testdata/expected-plans/q14.txt | 63 + testdata/expected-plans/q16.txt | 113 + testdata/expected-plans/q17.txt | 100 + testdata/expected-plans/q18.txt | 119 ++ testdata/expected-plans/q19.txt | 76 + testdata/expected-plans/q2.txt | 274 +++ testdata/expected-plans/q20.txt | 166 ++ testdata/expected-plans/q21.txt | 194 ++ testdata/expected-plans/q22.txt | 114 + testdata/expected-plans/q3.txt | 114 + testdata/expected-plans/q4.txt | 81 + testdata/expected-plans/q5.txt | 182 ++ testdata/expected-plans/q6.txt | 38 + testdata/expected-plans/q7.txt | 193 ++ testdata/expected-plans/q8.txt | 240 +++ testdata/expected-plans/q9.txt | 174 ++ testdata/queries/q1.sql | 23 + testdata/queries/q10.sql | 33 + testdata/queries/q11.sql | 29 + testdata/queries/q12.sql | 30 + testdata/queries/q13.sql | 22 + testdata/queries/q14.sql | 15 + testdata/queries/q15.sql | 33 + testdata/queries/q16.sql | 32 + testdata/queries/q17.sql | 19 + testdata/queries/q18.sql | 34 + testdata/queries/q19.sql | 37 + testdata/queries/q2.sql | 45 + testdata/queries/q20.sql | 39 + testdata/queries/q21.sql | 41 + testdata/queries/q22.sql | 39 + testdata/queries/q3.sql | 24 + testdata/queries/q4.sql | 23 + testdata/queries/q5.sql | 26 + testdata/queries/q6.sql | 11 + testdata/queries/q7.sql | 41 + testdata/queries/q8.sql | 39 + testdata/queries/q9.sql | 34 + 81 files changed, 11943 insertions(+), 1 deletion(-) create mode 100644 .github/workflows/rust.yml create mode 100644 .gitignore create mode 100644 Cargo.lock create mode 100644 Cargo.toml create mode 100644 LICENSE create mode 100644 NOTICE create mode 100644 build.rs create mode 100644 docs/README.md create mode 100644 docs/sqlbench-h-per-query.png create mode 100644 docs/sqlbench-h-total.png create mode 100644 examples/tips.csv create mode 100644 examples/tips.parquet create mode 100644 examples/tips.py create mode 100644 pyproject.toml create mode 100644 raysql/__init__.py create mode 100644 raysql/context.py create mode 100644 raysql/main.py create mode 100644 raysql/ray_utils.py create mode 100644 raysql/tests/test_context.py create mode 100644 requirements-in.txt create mode 100644 src/context.rs create mode 100644 src/lib.rs create mode 100644 src/planner.rs create mode 100644 src/proto/datafusion.proto create mode 100644 src/proto/generated/mod.rs create mode 100644 src/proto/generated/protobuf.rs create mode 100644 src/proto/mod.rs create mode 100644 src/proto/raysql.proto create mode 100644 src/query_stage.rs create mode 100644 src/shuffle/codec.rs create mode 100644 src/shuffle/mod.rs create mode 100644 src/shuffle/ray_shuffle/mod.rs create mode 100644 src/shuffle/ray_shuffle/reader.rs create mode 100644 src/shuffle/ray_shuffle/writer.rs create mode 100644 src/shuffle/reader.rs create mode 100644 src/shuffle/writer.rs create mode 100644 src/utils.rs create mode 100644 testdata/expected-plans/q1.txt create mode 100644 testdata/expected-plans/q10.txt create mode 100644 testdata/expected-plans/q11.txt create mode 100644 testdata/expected-plans/q12.txt create mode 100644 testdata/expected-plans/q13.txt create mode 100644 testdata/expected-plans/q14.txt create mode 100644 testdata/expected-plans/q16.txt create mode 100644 testdata/expected-plans/q17.txt create mode 100644 testdata/expected-plans/q18.txt create mode 100644 testdata/expected-plans/q19.txt create mode 100644 testdata/expected-plans/q2.txt create mode 100644 testdata/expected-plans/q20.txt create mode 100644 testdata/expected-plans/q21.txt create mode 100644 testdata/expected-plans/q22.txt create mode 100644 testdata/expected-plans/q3.txt create mode 100644 testdata/expected-plans/q4.txt create mode 100644 testdata/expected-plans/q5.txt create mode 100644 testdata/expected-plans/q6.txt create mode 100644 testdata/expected-plans/q7.txt create mode 100644 testdata/expected-plans/q8.txt create mode 100644 testdata/expected-plans/q9.txt create mode 100644 testdata/queries/q1.sql create mode 100644 testdata/queries/q10.sql create mode 100644 testdata/queries/q11.sql create mode 100644 testdata/queries/q12.sql create mode 100644 testdata/queries/q13.sql create mode 100644 testdata/queries/q14.sql create mode 100644 testdata/queries/q15.sql create mode 100644 testdata/queries/q16.sql create mode 100644 testdata/queries/q17.sql create mode 100644 testdata/queries/q18.sql create mode 100644 testdata/queries/q19.sql create mode 100644 testdata/queries/q2.sql create mode 100644 testdata/queries/q20.sql create mode 100644 testdata/queries/q21.sql create mode 100644 testdata/queries/q22.sql create mode 100644 testdata/queries/q3.sql create mode 100644 testdata/queries/q4.sql create mode 100644 testdata/queries/q5.sql create mode 100644 testdata/queries/q6.sql create mode 100644 testdata/queries/q7.sql create mode 100644 testdata/queries/q8.sql create mode 100644 testdata/queries/q9.sql diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml new file mode 100644 index 0000000..66d4aeb --- /dev/null +++ b/.github/workflows/rust.yml @@ -0,0 +1,25 @@ +name: Rust + +on: + push: + branches: [ "main" ] + pull_request: + branches: [ "main" ] + +env: + CARGO_TERM_COLOR: always + +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 diff --git a/.gitignore b/.gitignore new file mode 100644 index 0000000..0c56375 --- /dev/null +++ b/.gitignore @@ -0,0 +1,7 @@ +.idea +target +__pycache__ +venv +*.so +*.log +results-sf* diff --git a/Cargo.lock b/Cargo.lock new file mode 100644 index 0000000..01fd6dc --- /dev/null +++ b/Cargo.lock @@ -0,0 +1,3268 @@ +# This file is automatically @generated by Cargo. +# It is not intended for manual editing. +version = 3 + +[[package]] +name = "adler" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" + +[[package]] +name = "adler32" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aae1277d39aeec15cb388266ecc24b11c80469deae6067e17a1a7aa9e5c1f234" + +[[package]] +name = "ahash" +version = "0.8.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c99f64d1e06488f620f932677e24bc6e2897582980441ae90a671415bd7ec2f" +dependencies = [ + "cfg-if", + "const-random", + "getrandom", + "once_cell", + "version_check", +] + +[[package]] +name = "aho-corasick" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "67fc08ce920c31afb70f013dcce1bfc3a3195de6a228474e45e1f145b36f8d04" +dependencies = [ + "memchr", +] + +[[package]] +name = "alloc-no-stdlib" +version = "2.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cc7bb162ec39d46ab1ca8c77bf72e890535becd1751bb45f64c597edb4c8c6b3" + +[[package]] +name = "alloc-stdlib" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94fb8275041c72129eb51b7d0322c29b8387a0386127718b096429201a5d6ece" +dependencies = [ + "alloc-no-stdlib", +] + +[[package]] +name = "allocator-api2" +version = "0.2.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0942ffc6dcaadf03badf6e6a2d0228460359d5e34b57ccdc720b7382dfbd5ec5" + +[[package]] +name = "android-tzdata" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e999941b234f3131b00bc13c22d06e8c5ff726d1b6318ac7eb276997bbb4fef0" + +[[package]] +name = "android_system_properties" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "819e7219dbd41043ac279b19830f2efc897156490d7fd6ea916720117ee66311" +dependencies = [ + "libc", +] + +[[package]] +name = "anyhow" +version = "1.0.71" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c7d0618f0e0b7e8ff11427422b64564d5fb0be1940354bfe2e0529b18a9d9b8" + +[[package]] +name = "apache-avro" +version = "0.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ceb7c683b2f8f40970b70e39ff8be514c95b96fcb9c4af87e1ed2cb2e10801a0" +dependencies = [ + "bzip2", + "crc32fast", + "digest", + "lazy_static", + "libflate", + "log", + "num-bigint", + "quad-rand", + "rand", + "regex-lite", + "serde", + "serde_json", + "snap", + "strum 0.25.0", + "strum_macros 0.25.2", + "thiserror", + "typed-builder", + "uuid", + "xz2", + "zstd 0.12.4", +] + +[[package]] +name = "arrayref" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6b4930d2cb77ce62f89ee5d5289b4ac049559b1c45539271f5ed4fdc7db34545" + +[[package]] +name = "arrayvec" +version = "0.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8da52d66c7071e2e3fa2a1e5c6d088fec47b593032b254f5e980de8ea54454d6" + +[[package]] +name = "arrow" +version = "48.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a8919668503a4f2d8b6da96fa7c16e93046bfb3412ffcfa1e5dc7d2e3adcb378" +dependencies = [ + "ahash", + "arrow-arith", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-csv", + "arrow-data", + "arrow-ipc", + "arrow-json", + "arrow-ord", + "arrow-row", + "arrow-schema", + "arrow-select", + "arrow-string", + "pyo3", +] + +[[package]] +name = "arrow-arith" +version = "48.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ef983914f477d4278b068f13b3224b7d19eb2b807ac9048544d3bfebdf2554c4" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "chrono", + "half", + "num", +] + +[[package]] +name = "arrow-array" +version = "48.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d6eaf89041fa5937940ae390294ece29e1db584f46d995608d6e5fe65a2e0e9b" +dependencies = [ + "ahash", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "chrono", + "chrono-tz", + "half", + "hashbrown 0.14.0", + "num", +] + +[[package]] +name = "arrow-buffer" +version = "48.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "55512d988c6fbd76e514fd3ff537ac50b0a675da5a245e4fdad77ecfd654205f" +dependencies = [ + "bytes", + "half", + "num", +] + +[[package]] +name = "arrow-cast" +version = "48.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "655ee51a2156ba5375931ce21c1b2494b1d9260e6dcdc6d4db9060c37dc3325b" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", + "chrono", + "comfy-table", + "half", + "lexical-core", + "num", +] + +[[package]] +name = "arrow-csv" +version = "48.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "258bb689997ad5b6660b3ce3638bd6b383d668ec555ed41ad7c6559cbb2e4f91" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-schema", + "chrono", + "csv", + "csv-core", + "lazy_static", + "lexical-core", + "regex", +] + +[[package]] +name = "arrow-data" +version = "48.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6dc2b9fec74763427e2e5575b8cc31ce96ba4c9b4eb05ce40e0616d9fad12461" +dependencies = [ + "arrow-buffer", + "arrow-schema", + "half", + "num", +] + +[[package]] +name = "arrow-ipc" +version = "48.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6eaa6ab203cc6d89b7eaa1ac781c1dfeef325454c5d5a0419017f95e6bafc03c" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-schema", + "flatbuffers", +] + +[[package]] +name = "arrow-json" +version = "48.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fb64e30d9b73f66fdc5c52d5f4cf69bbf03d62f64ffeafa0715590a5320baed7" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-schema", + "chrono", + "half", + "indexmap 2.0.0", + "lexical-core", + "num", + "serde", + "serde_json", +] + +[[package]] +name = "arrow-ord" +version = "48.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9a818951c0d11c428dda03e908175969c262629dd20bd0850bd6c7a8c3bfe48" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", + "half", + "num", +] + +[[package]] +name = "arrow-row" +version = "48.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a5d664318bc05f930559fc088888f0f7174d3c5bc888c0f4f9ae8f23aa398ba3" +dependencies = [ + "ahash", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "half", + "hashbrown 0.14.0", +] + +[[package]] +name = "arrow-schema" +version = "48.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aaf4d737bba93da59f16129bec21e087aed0be84ff840e74146d4703879436cb" +dependencies = [ + "bitflags 2.2.1", +] + +[[package]] +name = "arrow-select" +version = "48.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "374c4c3b812ecc2118727b892252a4a4308f87a8aca1dbf09f3ce4bc578e668a" +dependencies = [ + "ahash", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "num", +] + +[[package]] +name = "arrow-string" +version = "48.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b15aed5624bb23da09142f58502b59c23f5bea607393298bb81dab1ce60fc769" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", + "num", + "regex", + "regex-syntax 0.8.2", +] + +[[package]] +name = "async-compression" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "62b74f44609f0f91493e3082d3734d98497e094777144380ea4db9f9905dd5b6" +dependencies = [ + "bzip2", + "flate2", + "futures-core", + "futures-io", + "memchr", + "pin-project-lite", + "tokio", + "xz2", + "zstd 0.12.4", + "zstd-safe 6.0.5+zstd.1.5.4", +] + +[[package]] +name = "async-trait" +version = "0.1.74" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a66537f1bb974b254c98ed142ff995236e81b9d0fe4db0575f46612cb15eb0f9" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.39", +] + +[[package]] +name = "autocfg" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" + +[[package]] +name = "base64" +version = "0.21.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a4a4ddaa51a5bc52a6948f74c06d20aaaddb71924eab79b8c97a8c556e942d6a" + +[[package]] +name = "bitflags" +version = "1.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" + +[[package]] +name = "bitflags" +version = "2.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "24a6904aef64d73cf10ab17ebace7befb918b82164785cb89907993be7f83813" + +[[package]] +name = "blake2" +version = "0.10.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "46502ad458c9a52b69d4d4d32775c788b7a1b85e8bc9d482d92250fc0e3f8efe" +dependencies = [ + "digest", +] + +[[package]] +name = "blake3" +version = "1.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "42ae2468a89544a466886840aa467a25b766499f4f04bf7d9fcd10ecee9fccef" +dependencies = [ + "arrayref", + "arrayvec", + "cc", + "cfg-if", + "constant_time_eq", + "digest", +] + +[[package]] +name = "block-buffer" +version = "0.10.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3078c7629b62d3f0439517fa394996acacc5cbc91c5a20d8c658e77abd503a71" +dependencies = [ + "generic-array", +] + +[[package]] +name = "brotli" +version = "3.3.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a1a0b1dbcc8ae29329621f8d4f0d835787c1c38bb1401979b49d13b0b305ff68" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", + "brotli-decompressor", +] + +[[package]] +name = "brotli-decompressor" +version = "2.3.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4b6561fd3f895a11e8f72af2cb7d22e08366bebc2b6b57f7744c4bda27034744" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", +] + +[[package]] +name = "bumpalo" +version = "3.12.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3c6ed94e98ecff0c12dd1b04c15ec0d7d9458ca8fe806cea6f12954efe74c63b" + +[[package]] +name = "byteorder" +version = "1.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "14c189c53d098945499cdfa7ecc63567cf3886b3332b312a5b4585d8d3a6a610" + +[[package]] +name = "bytes" +version = "1.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "89b2fd2a0dcf38d7971e2194b6b6eebab45ae01067456a7fd93d5547a61b70be" + +[[package]] +name = "bzip2" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bdb116a6ef3f6c3698828873ad02c3014b3c85cadb88496095628e3ef1e347f8" +dependencies = [ + "bzip2-sys", + "libc", +] + +[[package]] +name = "bzip2-sys" +version = "0.1.11+1.0.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "736a955f3fa7875102d57c82b8cac37ec45224a07fd32d58f9f7a186b6cd4cdc" +dependencies = [ + "cc", + "libc", + "pkg-config", +] + +[[package]] +name = "cc" +version = "1.0.79" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "50d30906286121d95be3d479533b458f87493b30a4b5f79a607db8f5d11aa91f" +dependencies = [ + "jobserver", +] + +[[package]] +name = "cfg-if" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" + +[[package]] +name = "chrono" +version = "0.4.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f2c685bad3eb3d45a01354cedb7d5faa66194d1d58ba6e267a8de788f79db38" +dependencies = [ + "android-tzdata", + "iana-time-zone", + "num-traits", + "serde", + "windows-targets 0.48.0", +] + +[[package]] +name = "chrono-tz" +version = "0.8.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cf9cc2b23599e6d7479755f3594285efb3f74a1bdca7a7374948bc831e23a552" +dependencies = [ + "chrono", + "chrono-tz-build", + "phf", +] + +[[package]] +name = "chrono-tz-build" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d9998fb9f7e9b2111641485bf8beb32f92945f97f92a3d061f744cfef335f751" +dependencies = [ + "parse-zoneinfo", + "phf", + "phf_codegen", +] + +[[package]] +name = "comfy-table" +version = "7.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ab77dbd8adecaf3f0db40581631b995f312a8a5ae3aa9993188bb8f23d83a5b" +dependencies = [ + "strum 0.24.1", + "strum_macros 0.24.3", + "unicode-width", +] + +[[package]] +name = "const-random" +version = "0.1.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "368a7a772ead6ce7e1de82bfb04c485f3db8ec744f72925af5735e29a22cc18e" +dependencies = [ + "const-random-macro", + "proc-macro-hack", +] + +[[package]] +name = "const-random-macro" +version = "0.1.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9d7d6ab3c3a2282db210df5f02c4dab6e0a7057af0fb7ebd4070f30fe05c0ddb" +dependencies = [ + "getrandom", + "once_cell", + "proc-macro-hack", + "tiny-keccak", +] + +[[package]] +name = "constant_time_eq" +version = "0.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "13418e745008f7349ec7e449155f419a61b92b58a99cc3616942b926825ec76b" + +[[package]] +name = "core-foundation-sys" +version = "0.8.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e496a50fda8aacccc86d7529e2c1e0892dbd0f898a6b5645b5561b89c3210efa" + +[[package]] +name = "core2" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b49ba7ef1ad6107f8824dbe97de947cbaac53c44e7f9756a1fba0d37c1eec505" +dependencies = [ + "memchr", +] + +[[package]] +name = "cpufeatures" +version = "0.2.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3e4c1eaa2012c47becbbad2ab175484c2a84d1185b566fb2cc5b8707343dfe58" +dependencies = [ + "libc", +] + +[[package]] +name = "crc32fast" +version = "1.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b540bd8bc810d3885c6ea91e2018302f68baba2129ab3e88f32389ee9370880d" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "crunchy" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" + +[[package]] +name = "crypto-common" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3" +dependencies = [ + "generic-array", + "typenum", +] + +[[package]] +name = "csv" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b015497079b9a9d69c02ad25de6c0a6edef051ea6360a327d0bd05802ef64ad" +dependencies = [ + "csv-core", + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "csv-core" +version = "0.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b2466559f260f48ad25fe6317b3c8dac77b5bdb5763ac7d9d6103530663bc90" +dependencies = [ + "memchr", +] + +[[package]] +name = "dary_heap" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7762d17f1241643615821a8455a0b2c3e803784b058693d990b11f2dce25a0ca" + +[[package]] +name = "dashmap" +version = "5.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "907076dfda823b0b36d2a1bb5f90c96660a5bbcd7729e10727f07858f22c4edc" +dependencies = [ + "cfg-if", + "hashbrown 0.12.3", + "lock_api", + "once_cell", + "parking_lot_core", +] + +[[package]] +name = "datafusion" +version = "33.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "676796427e638d85e9eadf13765705212be60b34f8fc5d3934d95184c63ca1b4" +dependencies = [ + "ahash", + "apache-avro", + "arrow", + "arrow-array", + "arrow-schema", + "async-compression", + "async-trait", + "bytes", + "bzip2", + "chrono", + "dashmap", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-optimizer", + "datafusion-physical-expr", + "datafusion-physical-plan", + "datafusion-sql", + "flate2", + "futures", + "glob", + "half", + "hashbrown 0.14.0", + "indexmap 2.0.0", + "itertools 0.11.0", + "log", + "num-traits", + "num_cpus", + "object_store", + "parking_lot", + "parquet", + "pin-project-lite", + "rand", + "sqlparser", + "tempfile", + "tokio", + "tokio-util", + "url", + "uuid", + "xz2", + "zstd 0.13.0", +] + +[[package]] +name = "datafusion-common" +version = "33.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "31e23b3d21a6531259d291bd20ce59282ea794bda1018b0a1e278c13cd52e50c" +dependencies = [ + "ahash", + "apache-avro", + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-schema", + "chrono", + "half", + "num_cpus", + "object_store", + "parquet", + "pyo3", + "sqlparser", +] + +[[package]] +name = "datafusion-execution" +version = "33.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4de1fd0d8db0f2b8e4f4121bfa1c7c09d3a5c08a0a65c2229cd849eb65cff855" +dependencies = [ + "arrow", + "chrono", + "dashmap", + "datafusion-common", + "datafusion-expr", + "futures", + "hashbrown 0.14.0", + "log", + "object_store", + "parking_lot", + "rand", + "tempfile", + "url", +] + +[[package]] +name = "datafusion-expr" +version = "33.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "18e227fe88bf6730cab378d0cd8fc4c6b2ea42bc7e414a8ea9feba7225932735" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "datafusion-common", + "sqlparser", + "strum 0.25.0", + "strum_macros 0.25.2", +] + +[[package]] +name = "datafusion-optimizer" +version = "33.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c6648e62ea7605b9bfcd87fdc9d67e579c3b9ac563a87734ae5fe6d79ee4547" +dependencies = [ + "arrow", + "async-trait", + "chrono", + "datafusion-common", + "datafusion-expr", + "datafusion-physical-expr", + "hashbrown 0.14.0", + "itertools 0.11.0", + "log", + "regex-syntax 0.8.2", +] + +[[package]] +name = "datafusion-physical-expr" +version = "33.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f32b8574add16a32411a9b3fb3844ac1fc09ab4e7be289f86fd56d620e4f2508" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-ord", + "arrow-schema", + "base64", + "blake2", + "blake3", + "chrono", + "datafusion-common", + "datafusion-expr", + "half", + "hashbrown 0.14.0", + "hex", + "indexmap 2.0.0", + "itertools 0.11.0", + "libc", + "log", + "md-5", + "paste", + "petgraph", + "rand", + "regex", + "sha2", + "unicode-segmentation", + "uuid", +] + +[[package]] +name = "datafusion-physical-plan" +version = "33.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "796abd77d5bfecd9e5275a99daf0ec45f5b3a793ec431349ce8211a67826fd22" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-schema", + "async-trait", + "chrono", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "futures", + "half", + "hashbrown 0.14.0", + "indexmap 2.0.0", + "itertools 0.11.0", + "log", + "once_cell", + "parking_lot", + "pin-project-lite", + "rand", + "tokio", + "uuid", +] + +[[package]] +name = "datafusion-proto" +version = "33.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "26de2592417beb20f73f29b131a04d7de14e2a6336c631554d611584b4306236" +dependencies = [ + "arrow", + "chrono", + "datafusion", + "datafusion-common", + "datafusion-expr", + "object_store", + "prost 0.12.2", +] + +[[package]] +name = "datafusion-python" +version = "33.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d28b42bdfdc35e25105878fcdf60abdc7faf79866344e23879671e4ad9c1ad5" +dependencies = [ + "async-trait", + "datafusion", + "datafusion-common", + "datafusion-expr", + "datafusion-optimizer", + "datafusion-sql", + "futures", + "mimalloc", + "object_store", + "parking_lot", + "prost 0.12.2", + "prost-types 0.12.2", + "pyo3", + "pyo3-build-config", + "rand", + "regex-syntax 0.8.2", + "syn 2.0.39", + "tokio", + "url", + "uuid", +] + +[[package]] +name = "datafusion-sql" +version = "33.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ced70b8a5648ba7b95c61fc512183c33287ffe2c9f22ffe22700619d7d48c84f" +dependencies = [ + "arrow", + "arrow-schema", + "datafusion-common", + "datafusion-expr", + "log", + "sqlparser", +] + +[[package]] +name = "digest" +version = "0.10.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" +dependencies = [ + "block-buffer", + "crypto-common", + "subtle", +] + +[[package]] +name = "doc-comment" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fea41bba32d969b513997752735605054bc0dfa92b4c56bf1189f2e174be7a10" + +[[package]] +name = "either" +version = "1.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7fcaabb2fef8c910e7f4c7ce9f67a1283a1715879a7c230ca9d6d1ae31f16d91" + +[[package]] +name = "encoding_rs" +version = "0.8.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "071a31f4ee85403370b58aca746f01041ede6f0da2730960ad001edc2b71b394" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "equivalent" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" + +[[package]] +name = "errno" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4bcfec3a70f97c962c307b2d2c56e358cf1d00b558d74262b5f929ee8cc7e73a" +dependencies = [ + "errno-dragonfly", + "libc", + "windows-sys 0.48.0", +] + +[[package]] +name = "errno-dragonfly" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aa68f1b12764fab894d2755d2518754e71b4fd80ecfb822714a1206c2aab39bf" +dependencies = [ + "cc", + "libc", +] + +[[package]] +name = "fastrand" +version = "1.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e51093e27b0797c359783294ca4f0a911c270184cb10f85783b118614a1501be" +dependencies = [ + "instant", +] + +[[package]] +name = "fixedbitset" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" + +[[package]] +name = "flatbuffers" +version = "23.1.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77f5399c2c9c50ae9418e522842ad362f61ee48b346ac106807bd355a8a7c619" +dependencies = [ + "bitflags 1.3.2", + "rustc_version", +] + +[[package]] +name = "flate2" +version = "1.0.26" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3b9429470923de8e8cbd4d2dc513535400b4b3fef0319fb5c4e1f520a7bef743" +dependencies = [ + "crc32fast", + "miniz_oxide", +] + +[[package]] +name = "fnv" +version = "1.0.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" + +[[package]] +name = "form_urlencoded" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a9c384f161156f5260c24a097c56119f9be8c798586aecc13afbcbe7b7e26bf8" +dependencies = [ + "percent-encoding", +] + +[[package]] +name = "futures" +version = "0.3.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23342abe12aba583913b2e62f22225ff9c950774065e4bfb61a19cd9770fec40" +dependencies = [ + "futures-channel", + "futures-core", + "futures-executor", + "futures-io", + "futures-sink", + "futures-task", + "futures-util", +] + +[[package]] +name = "futures-channel" +version = "0.3.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "955518d47e09b25bbebc7a18df10b81f0c766eaf4c4f1cccef2fca5f2a4fb5f2" +dependencies = [ + "futures-core", + "futures-sink", +] + +[[package]] +name = "futures-core" +version = "0.3.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4bca583b7e26f571124fe5b7561d49cb2868d79116cfa0eefce955557c6fee8c" + +[[package]] +name = "futures-executor" +version = "0.3.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ccecee823288125bd88b4d7f565c9e58e41858e47ab72e8ea2d64e93624386e0" +dependencies = [ + "futures-core", + "futures-task", + "futures-util", +] + +[[package]] +name = "futures-io" +version = "0.3.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4fff74096e71ed47f8e023204cfd0aa1289cd54ae5430a9523be060cdb849964" + +[[package]] +name = "futures-macro" +version = "0.3.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "89ca545a94061b6365f2c7355b4b32bd20df3ff95f02da9329b34ccc3bd6ee72" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.39", +] + +[[package]] +name = "futures-sink" +version = "0.3.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f43be4fe21a13b9781a69afa4985b0f6ee0e1afab2c6f454a8cf30e2b2237b6e" + +[[package]] +name = "futures-task" +version = "0.3.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "76d3d132be6c0e6aa1534069c705a74a5997a356c0dc2f86a47765e5617c5b65" + +[[package]] +name = "futures-util" +version = "0.3.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "26b01e40b772d54cf6c6d721c1d1abd0647a0106a12ecaa1c186273392a69533" +dependencies = [ + "futures-channel", + "futures-core", + "futures-io", + "futures-macro", + "futures-sink", + "futures-task", + "memchr", + "pin-project-lite", + "pin-utils", + "slab", +] + +[[package]] +name = "generic-array" +version = "0.14.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85649ca51fd72272d7821adaf274ad91c288277713d9c18820d8499a7ff69e9a" +dependencies = [ + "typenum", + "version_check", +] + +[[package]] +name = "getrandom" +version = "0.2.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c85e1d9ab2eadba7e5040d4e09cbd6d072b76a557ad64e797c2cb9d4da21d7e4" +dependencies = [ + "cfg-if", + "libc", + "wasi", +] + +[[package]] +name = "glob" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" + +[[package]] +name = "h2" +version = "0.3.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d357c7ae988e7d2182f7d7871d0b963962420b0678b0997ce7de72001aeab782" +dependencies = [ + "bytes", + "fnv", + "futures-core", + "futures-sink", + "futures-util", + "http", + "indexmap 1.9.3", + "slab", + "tokio", + "tokio-util", + "tracing", +] + +[[package]] +name = "half" +version = "2.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "02b4af3693f1b705df946e9fe5631932443781d0aabb423b62fcd4d73f6d2fd0" +dependencies = [ + "crunchy", + "num-traits", +] + +[[package]] +name = "hashbrown" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" + +[[package]] +name = "hashbrown" +version = "0.13.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43a3c133739dddd0d2990f9a4bdf8eb4b21ef50e4851ca85ab661199821d510e" +dependencies = [ + "ahash", +] + +[[package]] +name = "hashbrown" +version = "0.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c6201b9ff9fd90a5a3bac2e56a830d0caa509576f0e503818ee82c181b3437a" +dependencies = [ + "ahash", + "allocator-api2", +] + +[[package]] +name = "heck" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" + +[[package]] +name = "hermit-abi" +version = "0.2.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ee512640fe35acbfb4bb779db6f0d80704c2cacfa2e39b601ef3e3f47d1ae4c7" +dependencies = [ + "libc", +] + +[[package]] +name = "hermit-abi" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fed44880c466736ef9a5c5b5facefb5ed0785676d0c02d612db14e54f0d84286" + +[[package]] +name = "hex" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" + +[[package]] +name = "http" +version = "0.2.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd6effc99afb63425aff9b05836f029929e345a6148a14b7ecd5ab67af944482" +dependencies = [ + "bytes", + "fnv", + "itoa", +] + +[[package]] +name = "http-body" +version = "0.4.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d5f38f16d184e36f2408a55281cd658ecbd3ca05cce6d6510a176eca393e26d1" +dependencies = [ + "bytes", + "http", + "pin-project-lite", +] + +[[package]] +name = "httparse" +version = "1.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d897f394bad6a705d5f4104762e116a75639e470d80901eed05a860a95cb1904" + +[[package]] +name = "httpdate" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c4a1e36c821dbe04574f602848a19f742f4fb3c98d40449f11bcad18d6b17421" + +[[package]] +name = "humantime" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" + +[[package]] +name = "hyper" +version = "0.14.26" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ab302d72a6f11a3b910431ff93aae7e773078c769f0a3ef15fb9ec692ed147d4" +dependencies = [ + "bytes", + "futures-channel", + "futures-core", + "futures-util", + "h2", + "http", + "http-body", + "httparse", + "httpdate", + "itoa", + "pin-project-lite", + "socket2", + "tokio", + "tower-service", + "tracing", + "want", +] + +[[package]] +name = "hyper-rustls" +version = "0.23.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1788965e61b367cd03a62950836d5cd41560c3577d90e40e0819373194d1661c" +dependencies = [ + "http", + "hyper", + "rustls", + "tokio", + "tokio-rustls", +] + +[[package]] +name = "iana-time-zone" +version = "0.1.56" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0722cd7114b7de04316e7ea5456a0bbb20e4adb46fd27a3697adb812cff0f37c" +dependencies = [ + "android_system_properties", + "core-foundation-sys", + "iana-time-zone-haiku", + "js-sys", + "wasm-bindgen", + "windows", +] + +[[package]] +name = "iana-time-zone-haiku" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f31827a206f56af32e590ba56d5d2d085f558508192593743f16b2306495269f" +dependencies = [ + "cc", +] + +[[package]] +name = "idna" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e14ddfc70884202db2244c223200c204c2bda1bc6e0998d11b5e024d657209e6" +dependencies = [ + "unicode-bidi", + "unicode-normalization", +] + +[[package]] +name = "indexmap" +version = "1.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" +dependencies = [ + "autocfg", + "hashbrown 0.12.3", +] + +[[package]] +name = "indexmap" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d5477fe2230a79769d8dc68e0eabf5437907c0457a5614a9e8dddb67f65eb65d" +dependencies = [ + "equivalent", + "hashbrown 0.14.0", +] + +[[package]] +name = "indoc" +version = "2.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e186cfbae8084e513daff4240b4797e342f988cecda4fb6c939150f96315fd8" + +[[package]] +name = "instant" +version = "0.1.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a5bbe824c507c5da5956355e86a746d82e0e1464f65d862cc5e71da70e94b2c" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "integer-encoding" +version = "3.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" + +[[package]] +name = "io-lifetimes" +version = "1.0.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c66c74d2ae7e79a5a8f7ac924adbe38ee42a859c6539ad869eb51f0b52dc220" +dependencies = [ + "hermit-abi 0.3.1", + "libc", + "windows-sys 0.48.0", +] + +[[package]] +name = "ipnet" +version = "2.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "12b6ee2129af8d4fb011108c73d99a1b83a85977f23b82460c0ae2e25bb4b57f" + +[[package]] +name = "itertools" +version = "0.10.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" +dependencies = [ + "either", +] + +[[package]] +name = "itertools" +version = "0.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b1c173a5686ce8bfa551b3563d0c2170bf24ca44da99c7ca4bfdab5418c3fe57" +dependencies = [ + "either", +] + +[[package]] +name = "itoa" +version = "1.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "453ad9f582a441959e5f0d088b02ce04cfe8d51a8eaf077f12ac6d3e94164ca6" + +[[package]] +name = "jobserver" +version = "0.1.26" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "936cfd212a0155903bcbc060e316fb6cc7cbf2e1907329391ebadc1fe0ce77c2" +dependencies = [ + "libc", +] + +[[package]] +name = "js-sys" +version = "0.3.62" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68c16e1bfd491478ab155fd8b4896b86f9ede344949b641e61501e07c2b8b4d5" +dependencies = [ + "wasm-bindgen", +] + +[[package]] +name = "lazy_static" +version = "1.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" + +[[package]] +name = "lexical-core" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2cde5de06e8d4c2faabc400238f9ae1c74d5412d03a7bd067645ccbc47070e46" +dependencies = [ + "lexical-parse-float", + "lexical-parse-integer", + "lexical-util", + "lexical-write-float", + "lexical-write-integer", +] + +[[package]] +name = "lexical-parse-float" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "683b3a5ebd0130b8fb52ba0bdc718cc56815b6a097e28ae5a6997d0ad17dc05f" +dependencies = [ + "lexical-parse-integer", + "lexical-util", + "static_assertions", +] + +[[package]] +name = "lexical-parse-integer" +version = "0.8.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d0994485ed0c312f6d965766754ea177d07f9c00c9b82a5ee62ed5b47945ee9" +dependencies = [ + "lexical-util", + "static_assertions", +] + +[[package]] +name = "lexical-util" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5255b9ff16ff898710eb9eb63cb39248ea8a5bb036bea8085b1a767ff6c4e3fc" +dependencies = [ + "static_assertions", +] + +[[package]] +name = "lexical-write-float" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "accabaa1c4581f05a3923d1b4cfd124c329352288b7b9da09e766b0668116862" +dependencies = [ + "lexical-util", + "lexical-write-integer", + "static_assertions", +] + +[[package]] +name = "lexical-write-integer" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e1b6f3d1f4422866b68192d62f77bc5c700bee84f3069f2469d7bc8c77852446" +dependencies = [ + "lexical-util", + "static_assertions", +] + +[[package]] +name = "libc" +version = "0.2.144" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b00cc1c228a6782d0f076e7b232802e0c5689d41bb5df366f2a6b6621cfdfe1" + +[[package]] +name = "libflate" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9f7d5654ae1795afc7ff76f4365c2c8791b0feb18e8996a96adad8ffd7c3b2bf" +dependencies = [ + "adler32", + "core2", + "crc32fast", + "dary_heap", + "libflate_lz77", +] + +[[package]] +name = "libflate_lz77" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be5f52fb8c451576ec6b79d3f4deb327398bc05bbdbd99021a6e77a4c855d524" +dependencies = [ + "core2", + "hashbrown 0.13.2", + "rle-decode-fast", +] + +[[package]] +name = "libm" +version = "0.2.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "348108ab3fba42ec82ff6e9564fc4ca0247bdccdc68dd8af9764bbc79c3c8ffb" + +[[package]] +name = "libmimalloc-sys" +version = "0.1.33" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f4ac0e912c8ef1b735e92369695618dc5b1819f5a7bf3f167301a3ba1cea515e" +dependencies = [ + "cc", + "libc", +] + +[[package]] +name = "linux-raw-sys" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ece97ea872ece730aed82664c424eb4c8291e1ff2480247ccf7409044bc6479f" + +[[package]] +name = "lock_api" +version = "0.4.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "435011366fe56583b16cf956f9df0095b405b82d76425bc8981c0e22e60ec4df" +dependencies = [ + "autocfg", + "scopeguard", +] + +[[package]] +name = "log" +version = "0.4.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b5e6163cb8c49088c2c36f57875e58ccd8c87c7427f7fbd50ea6710b2f3f2e8f" + +[[package]] +name = "lz4_flex" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3ea9b256699eda7b0387ffbc776dd625e28bde3918446381781245b7a50349d8" +dependencies = [ + "twox-hash", +] + +[[package]] +name = "lzma-sys" +version = "0.1.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5fda04ab3764e6cde78b9974eec4f779acaba7c4e84b36eca3cf77c581b85d27" +dependencies = [ + "cc", + "libc", + "pkg-config", +] + +[[package]] +name = "md-5" +version = "0.10.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6365506850d44bff6e2fbcb5176cf63650e48bd45ef2fe2665ae1570e0f4b9ca" +dependencies = [ + "digest", +] + +[[package]] +name = "memchr" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2dffe52ecf27772e601905b7522cb4ef790d2cc203488bbd0e2fe85fcb74566d" + +[[package]] +name = "memoffset" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a634b1c61a95585bd15607c6ab0c4e5b226e695ff2800ba0cdccddf208c406c" +dependencies = [ + "autocfg", +] + +[[package]] +name = "mimalloc" +version = "0.1.37" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4e2894987a3459f3ffb755608bd82188f8ed00d0ae077f1edea29c068d639d98" +dependencies = [ + "libmimalloc-sys", +] + +[[package]] +name = "mime" +version = "0.3.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6877bb514081ee2a7ff5ef9de3281f14a4dd4bceac4c09388074a6b5df8a139a" + +[[package]] +name = "miniz_oxide" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e7810e0be55b428ada41041c41f32c9f1a42817901b4ccf45fa3d4b6561e74c7" +dependencies = [ + "adler", +] + +[[package]] +name = "mio" +version = "0.8.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b9d9a46eff5b4ff64b45a9e316a6d1e0bc719ef429cbec4dc630684212bfdf9" +dependencies = [ + "libc", + "log", + "wasi", + "windows-sys 0.45.0", +] + +[[package]] +name = "multimap" +version = "0.8.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5ce46fe64a9d73be07dcbe690a38ce1b293be448fd8ce1e6c1b8062c9f72c6a" + +[[package]] +name = "num" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b05180d69e3da0e530ba2a1dae5110317e49e3b7f3d41be227dc5f92e49ee7af" +dependencies = [ + "num-bigint", + "num-complex", + "num-integer", + "num-iter", + "num-rational", + "num-traits", +] + +[[package]] +name = "num-bigint" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "608e7659b5c3d7cba262d894801b9ec9d00de989e8a82bd4bef91d08da45cdc0" +dependencies = [ + "autocfg", + "num-integer", + "num-traits", +] + +[[package]] +name = "num-complex" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "02e0d21255c828d6f128a1e41534206671e8c3ea0c62f32291e808dc82cff17d" +dependencies = [ + "num-traits", +] + +[[package]] +name = "num-integer" +version = "0.1.45" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "225d3389fb3509a24c93f5c29eb6bde2586b98d9f016636dff58d7c6f7569cd9" +dependencies = [ + "autocfg", + "num-traits", +] + +[[package]] +name = "num-iter" +version = "0.1.43" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7d03e6c028c5dc5cac6e2dec0efda81fc887605bb3d884578bb6d6bf7514e252" +dependencies = [ + "autocfg", + "num-integer", + "num-traits", +] + +[[package]] +name = "num-rational" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0638a1c9d0a3c0914158145bc76cff373a75a627e6ecbfb71cbe6f453a5a19b0" +dependencies = [ + "autocfg", + "num-bigint", + "num-integer", + "num-traits", +] + +[[package]] +name = "num-traits" +version = "0.2.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "39e3200413f237f41ab11ad6d161bc7239c84dcb631773ccd7de3dfe4b5c267c" +dependencies = [ + "autocfg", + "libm", +] + +[[package]] +name = "num_cpus" +version = "1.15.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0fac9e2da13b5eb447a6ce3d392f23a29d8694bff781bf03a16cd9ac8697593b" +dependencies = [ + "hermit-abi 0.2.6", + "libc", +] + +[[package]] +name = "object_store" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f930c88a43b1c3f6e776dfe495b4afab89882dbc81530c632db2ed65451ebcb4" +dependencies = [ + "async-trait", + "base64", + "bytes", + "chrono", + "futures", + "humantime", + "hyper", + "itertools 0.11.0", + "parking_lot", + "percent-encoding", + "quick-xml", + "rand", + "reqwest", + "ring", + "rustls-pemfile", + "serde", + "serde_json", + "snafu", + "tokio", + "tracing", + "url", + "walkdir", +] + +[[package]] +name = "once_cell" +version = "1.18.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d" + +[[package]] +name = "ordered-float" +version = "2.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7940cf2ca942593318d07fcf2596cdca60a85c9e7fab408a5e21a4f9dcd40d87" +dependencies = [ + "num-traits", +] + +[[package]] +name = "parking_lot" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3742b2c103b9f06bc9fff0a37ff4912935851bee6d36f3c02bcc755bcfec228f" +dependencies = [ + "lock_api", + "parking_lot_core", +] + +[[package]] +name = "parking_lot_core" +version = "0.9.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9069cbb9f99e3a5083476ccb29ceb1de18b9118cafa53e90c9551235de2b9521" +dependencies = [ + "cfg-if", + "libc", + "redox_syscall 0.2.16", + "smallvec", + "windows-sys 0.45.0", +] + +[[package]] +name = "parquet" +version = "48.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6bfe55df96e3f02f11bf197ae37d91bb79801631f82f6195dd196ef521df3597" +dependencies = [ + "ahash", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-ipc", + "arrow-schema", + "arrow-select", + "base64", + "brotli", + "bytes", + "chrono", + "flate2", + "futures", + "hashbrown 0.14.0", + "lz4_flex", + "num", + "num-bigint", + "object_store", + "paste", + "seq-macro", + "snap", + "thrift", + "tokio", + "twox-hash", + "zstd 0.13.0", +] + +[[package]] +name = "parse-zoneinfo" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c705f256449c60da65e11ff6626e0c16a0a0b96aaa348de61376b249bc340f41" +dependencies = [ + "regex", +] + +[[package]] +name = "paste" +version = "1.0.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9f746c4065a8fa3fe23974dd82f15431cc8d40779821001404d10d2e79ca7d79" + +[[package]] +name = "percent-encoding" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "478c572c3d73181ff3c2539045f6eb99e5491218eae919370993b890cdbdd98e" + +[[package]] +name = "petgraph" +version = "0.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4dd7d28ee937e54fe3080c91faa1c3a46c06de6252988a7f4592ba2310ef22a4" +dependencies = [ + "fixedbitset", + "indexmap 1.9.3", +] + +[[package]] +name = "phf" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "928c6535de93548188ef63bb7c4036bd415cd8f36ad25af44b9789b2ee72a48c" +dependencies = [ + "phf_shared", +] + +[[package]] +name = "phf_codegen" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a56ac890c5e3ca598bbdeaa99964edb5b0258a583a9eb6ef4e89fc85d9224770" +dependencies = [ + "phf_generator", + "phf_shared", +] + +[[package]] +name = "phf_generator" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b1181c94580fa345f50f19d738aaa39c0ed30a600d95cb2d3e23f94266f14fbf" +dependencies = [ + "phf_shared", + "rand", +] + +[[package]] +name = "phf_shared" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e1fb5f6f826b772a8d4c0394209441e7d37cbbb967ae9c7e0e8134365c9ee676" +dependencies = [ + "siphasher", +] + +[[package]] +name = "pin-project-lite" +version = "0.2.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e0a7ae3ac2f1173085d398531c705756c94a4c56843785df85a60c1a0afac116" + +[[package]] +name = "pin-utils" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" + +[[package]] +name = "pkg-config" +version = "0.3.27" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "26072860ba924cbfa98ea39c8c19b4dd6a4a25423dbdf219c1eca91aa0cf6964" + +[[package]] +name = "ppv-lite86" +version = "0.2.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" + +[[package]] +name = "prettyplease" +version = "0.1.25" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6c8646e95016a7a6c4adea95bafa8a16baab64b583356217f2c85db4a39d9a86" +dependencies = [ + "proc-macro2", + "syn 1.0.109", +] + +[[package]] +name = "proc-macro-hack" +version = "0.5.20+deprecated" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc375e1527247fe1a97d8b7156678dfe7c1af2fc075c9a4db3690ecd2a148068" + +[[package]] +name = "proc-macro2" +version = "1.0.69" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "134c189feb4956b20f6f547d2cf727d4c0fe06722b20a0eec87ed445a97f92da" +dependencies = [ + "unicode-ident", +] + +[[package]] +name = "prost" +version = "0.11.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b82eaa1d779e9a4bc1c3217db8ffbeabaae1dca241bf70183242128d48681cd" +dependencies = [ + "bytes", + "prost-derive 0.11.9", +] + +[[package]] +name = "prost" +version = "0.12.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a5a410fc7882af66deb8d01d01737353cf3ad6204c408177ba494291a626312" +dependencies = [ + "bytes", + "prost-derive 0.12.2", +] + +[[package]] +name = "prost-build" +version = "0.11.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "119533552c9a7ffacc21e099c24a0ac8bb19c2a2a3f363de84cd9b844feab270" +dependencies = [ + "bytes", + "heck", + "itertools 0.10.5", + "lazy_static", + "log", + "multimap", + "petgraph", + "prettyplease", + "prost 0.11.9", + "prost-types 0.11.9", + "regex", + "syn 1.0.109", + "tempfile", + "which", +] + +[[package]] +name = "prost-derive" +version = "0.11.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5d2d8d10f3c6ded6da8b05b5fb3b8a5082514344d56c9f871412d29b4e075b4" +dependencies = [ + "anyhow", + "itertools 0.10.5", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "prost-derive" +version = "0.12.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "065717a5dfaca4a83d2fe57db3487b311365200000551d7a364e715dbf4346bc" +dependencies = [ + "anyhow", + "itertools 0.11.0", + "proc-macro2", + "quote", + "syn 2.0.39", +] + +[[package]] +name = "prost-types" +version = "0.11.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "213622a1460818959ac1181aaeb2dc9c7f63df720db7d788b3e24eacd1983e13" +dependencies = [ + "prost 0.11.9", +] + +[[package]] +name = "prost-types" +version = "0.12.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8339f32236f590281e2f6368276441394fcd1b2133b549cc895d0ae80f2f9a52" +dependencies = [ + "prost 0.12.2", +] + +[[package]] +name = "pyo3" +version = "0.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "04e8453b658fe480c3e70c8ed4e3d3ec33eb74988bd186561b0cc66b85c3bc4b" +dependencies = [ + "cfg-if", + "indoc", + "libc", + "memoffset", + "parking_lot", + "pyo3-build-config", + "pyo3-ffi", + "pyo3-macros", + "unindent", +] + +[[package]] +name = "pyo3-build-config" +version = "0.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a96fe70b176a89cff78f2fa7b3c930081e163d5379b4dcdf993e3ae29ca662e5" +dependencies = [ + "once_cell", + "target-lexicon", +] + +[[package]] +name = "pyo3-ffi" +version = "0.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "214929900fd25e6604661ed9cf349727c8920d47deff196c4e28165a6ef2a96b" +dependencies = [ + "libc", + "pyo3-build-config", +] + +[[package]] +name = "pyo3-macros" +version = "0.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dac53072f717aa1bfa4db832b39de8c875b7c7af4f4a6fe93cdbf9264cf8383b" +dependencies = [ + "proc-macro2", + "pyo3-macros-backend", + "quote", + "syn 2.0.39", +] + +[[package]] +name = "pyo3-macros-backend" +version = "0.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7774b5a8282bd4f25f803b1f0d945120be959a36c72e08e7cd031c792fdfd424" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 2.0.39", +] + +[[package]] +name = "quad-rand" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "658fa1faf7a4cc5f057c9ee5ef560f717ad9d8dc66d975267f709624d6e1ab88" + +[[package]] +name = "quick-xml" +version = "0.30.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eff6510e86862b57b210fd8cbe8ed3f0d7d600b9c2863cd4549a2e033c66e956" +dependencies = [ + "memchr", + "serde", +] + +[[package]] +name = "quote" +version = "1.0.33" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5267fca4496028628a95160fc423a33e8b2e6af8a5302579e322e4b520293cae" +dependencies = [ + "proc-macro2", +] + +[[package]] +name = "rand" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" +dependencies = [ + "libc", + "rand_chacha", + "rand_core", +] + +[[package]] +name = "rand_chacha" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" +dependencies = [ + "ppv-lite86", + "rand_core", +] + +[[package]] +name = "rand_core" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" +dependencies = [ + "getrandom", +] + +[[package]] +name = "raysql" +version = "0.6.0" +dependencies = [ + "datafusion", + "datafusion-proto", + "datafusion-python", + "futures", + "glob", + "log", + "prost 0.12.2", + "prost-types 0.12.2", + "pyo3", + "rustc_version", + "tokio", + "tonic-build", + "uuid", +] + +[[package]] +name = "redox_syscall" +version = "0.2.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fb5a58c1855b4b6819d59012155603f0b22ad30cad752600aadfcb695265519a" +dependencies = [ + "bitflags 1.3.2", +] + +[[package]] +name = "redox_syscall" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "567664f262709473930a4bf9e51bf2ebf3348f2e748ccc50dea20646858f8f29" +dependencies = [ + "bitflags 1.3.2", +] + +[[package]] +name = "regex" +version = "1.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af83e617f331cc6ae2da5443c602dfa5af81e517212d9d611a5b3ba1777b5370" +dependencies = [ + "aho-corasick", + "memchr", + "regex-syntax 0.7.1", +] + +[[package]] +name = "regex-lite" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "30b661b2f27137bdbc16f00eda72866a92bb28af1753ffbd56744fb6e2e9cd8e" + +[[package]] +name = "regex-syntax" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a5996294f19bd3aae0453a862ad728f60e6600695733dd5df01da90c54363a3c" + +[[package]] +name = "regex-syntax" +version = "0.8.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c08c74e62047bb2de4ff487b251e4a92e24f48745648451635cec7d591162d9f" + +[[package]] +name = "reqwest" +version = "0.11.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "13293b639a097af28fc8a90f22add145a9c954e49d77da06263d58cf44d5fb91" +dependencies = [ + "base64", + "bytes", + "encoding_rs", + "futures-core", + "futures-util", + "h2", + "http", + "http-body", + "hyper", + "hyper-rustls", + "ipnet", + "js-sys", + "log", + "mime", + "once_cell", + "percent-encoding", + "pin-project-lite", + "rustls", + "rustls-pemfile", + "serde", + "serde_json", + "serde_urlencoded", + "tokio", + "tokio-rustls", + "tokio-util", + "tower-service", + "url", + "wasm-bindgen", + "wasm-bindgen-futures", + "wasm-streams", + "web-sys", + "webpki-roots", + "winreg", +] + +[[package]] +name = "ring" +version = "0.16.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3053cf52e236a3ed746dfc745aa9cacf1b791d846bdaf412f60a8d7d6e17c8fc" +dependencies = [ + "cc", + "libc", + "once_cell", + "spin", + "untrusted", + "web-sys", + "winapi", +] + +[[package]] +name = "rle-decode-fast" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3582f63211428f83597b51b2ddb88e2a91a9d52d12831f9d08f5e624e8977422" + +[[package]] +name = "rustc_version" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bfa0f585226d2e68097d4f95d113b15b83a82e819ab25717ec0590d9584ef366" +dependencies = [ + "semver", +] + +[[package]] +name = "rustix" +version = "0.37.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "acf8729d8542766f1b2cf77eb034d52f40d375bb8b615d0b147089946e16613d" +dependencies = [ + "bitflags 1.3.2", + "errno", + "io-lifetimes", + "libc", + "linux-raw-sys", + "windows-sys 0.48.0", +] + +[[package]] +name = "rustls" +version = "0.20.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fff78fc74d175294f4e83b28343315ffcfb114b156f0185e9741cb5570f50e2f" +dependencies = [ + "log", + "ring", + "sct", + "webpki", +] + +[[package]] +name = "rustls-pemfile" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d194b56d58803a43635bdc398cd17e383d6f71f9182b9a192c127ca42494a59b" +dependencies = [ + "base64", +] + +[[package]] +name = "rustversion" +version = "1.0.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4f3208ce4d8448b3f3e7d168a73f5e0c43a61e32930de3bceeccedb388b6bf06" + +[[package]] +name = "ryu" +version = "1.0.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f91339c0467de62360649f8d3e185ca8de4224ff281f66000de5eb2a77a79041" + +[[package]] +name = "same-file" +version = "1.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93fc1dc3aaa9bfed95e02e6eadabb4baf7e3078b0bd1b4d7b6b0b68378900502" +dependencies = [ + "winapi-util", +] + +[[package]] +name = "scopeguard" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d29ab0c6d3fc0ee92fe66e2d99f700eab17a8d57d1c1d3b748380fb20baa78cd" + +[[package]] +name = "sct" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d53dcdb7c9f8158937a7981b48accfd39a43af418591a5d008c7b22b5e1b7ca4" +dependencies = [ + "ring", + "untrusted", +] + +[[package]] +name = "semver" +version = "1.0.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bebd363326d05ec3e2f532ab7660680f3b02130d780c299bca73469d521bc0ed" + +[[package]] +name = "seq-macro" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6b44e8fc93a14e66336d230954dda83d18b4605ccace8fe09bc7514a71ad0bc" + +[[package]] +name = "serde" +version = "1.0.192" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bca2a08484b285dcb282d0f67b26cadc0df8b19f8c12502c13d966bf9482f001" +dependencies = [ + "serde_derive", +] + +[[package]] +name = "serde_derive" +version = "1.0.192" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d6c7207fbec9faa48073f3e3074cbe553af6ea512d7c21ba46e434e70ea9fbc1" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.39", +] + +[[package]] +name = "serde_json" +version = "1.0.108" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3d1c7e3eac408d115102c4c24ad393e0821bb3a5df4d506a80f85f7a742a526b" +dependencies = [ + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "serde_urlencoded" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d3491c14715ca2294c4d6a88f15e84739788c1d030eed8c110436aafdaa2f3fd" +dependencies = [ + "form_urlencoded", + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "sha2" +version = "0.10.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "82e6b795fe2e3b1e845bafcb27aa35405c4d47cdfc92af5fc8d3002f76cebdc0" +dependencies = [ + "cfg-if", + "cpufeatures", + "digest", +] + +[[package]] +name = "siphasher" +version = "0.3.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7bd3e3206899af3f8b12af284fafc038cc1dc2b41d1b89dd17297221c5d225de" + +[[package]] +name = "slab" +version = "0.4.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6528351c9bc8ab22353f9d776db39a20288e8d6c37ef8cfe3317cf875eecfc2d" +dependencies = [ + "autocfg", +] + +[[package]] +name = "smallvec" +version = "1.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a507befe795404456341dfab10cef66ead4c041f62b8b11bbb92bffe5d0953e0" + +[[package]] +name = "snafu" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cb0656e7e3ffb70f6c39b3c2a86332bb74aa3c679da781642590f3c1118c5045" +dependencies = [ + "doc-comment", + "snafu-derive", +] + +[[package]] +name = "snafu-derive" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "475b3bbe5245c26f2d8a6f62d67c1f30eb9fffeccee721c45d162c3ebbdf81b2" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "snap" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5e9f0ab6ef7eb7353d9119c170a436d1bf248eea575ac42d19d12f4e34130831" + +[[package]] +name = "socket2" +version = "0.4.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "64a4a911eed85daf18834cfaa86a79b7d266ff93ff5ba14005426219480ed662" +dependencies = [ + "libc", + "winapi", +] + +[[package]] +name = "spin" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6e63cff320ae2c57904679ba7cb63280a3dc4613885beafb148ee7bf9aa9042d" + +[[package]] +name = "sqlparser" +version = "0.39.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "743b4dc2cbde11890ccb254a8fc9d537fa41b36da00de2a1c5e9848c9bc42bd7" +dependencies = [ + "log", + "sqlparser_derive", +] + +[[package]] +name = "sqlparser_derive" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "55fe75cb4a364c7f7ae06c7dbbc8d84bddd85d6cdf9975963c3935bc1991761e" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "static_assertions" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" + +[[package]] +name = "strum" +version = "0.24.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "063e6045c0e62079840579a7e47a355ae92f60eb74daaf156fb1e84ba164e63f" + +[[package]] +name = "strum" +version = "0.25.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "290d54ea6f91c969195bdbcd7442c8c2a2ba87da8bf60a7ee86a235d4bc1e125" +dependencies = [ + "strum_macros 0.25.2", +] + +[[package]] +name = "strum_macros" +version = "0.24.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e385be0d24f186b4ce2f9982191e7101bb737312ad61c1f2f984f34bcf85d59" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "rustversion", + "syn 1.0.109", +] + +[[package]] +name = "strum_macros" +version = "0.25.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ad8d03b598d3d0fff69bf533ee3ef19b8eeb342729596df84bcc7e1f96ec4059" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "rustversion", + "syn 2.0.39", +] + +[[package]] +name = "subtle" +version = "2.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6bdef32e8150c2a081110b42772ffe7d7c9032b606bc226c8260fd97e0976601" + +[[package]] +name = "syn" +version = "1.0.109" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "syn" +version = "2.0.39" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23e78b90f2fcf45d3e842032ce32e3f2d1545ba6636271dcbf24fa306d87be7a" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "target-lexicon" +version = "0.12.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fd1ba337640d60c3e96bc6f0638a939b9c9a7f2c316a1598c279828b3d1dc8c5" + +[[package]] +name = "tempfile" +version = "3.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b9fbec84f381d5795b08656e4912bec604d162bff9291d6189a78f4c8ab87998" +dependencies = [ + "cfg-if", + "fastrand", + "redox_syscall 0.3.5", + "rustix", + "windows-sys 0.45.0", +] + +[[package]] +name = "thiserror" +version = "1.0.50" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9a7210f5c9a7156bb50aa36aed4c95afb51df0df00713949448cf9e97d382d2" +dependencies = [ + "thiserror-impl", +] + +[[package]] +name = "thiserror-impl" +version = "1.0.50" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "266b2e40bc00e5a6c09c3584011e08b06f123c00362c92b975ba9843aaaa14b8" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.39", +] + +[[package]] +name = "thrift" +version = "0.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7e54bc85fc7faa8bc175c4bab5b92ba8d9a3ce893d0e9f42cc455c8ab16a9e09" +dependencies = [ + "byteorder", + "integer-encoding", + "ordered-float", +] + +[[package]] +name = "tiny-keccak" +version = "2.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c9d3793400a45f954c52e73d068316d76b6f4e36977e3fcebb13a2721e80237" +dependencies = [ + "crunchy", +] + +[[package]] +name = "tinyvec" +version = "1.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87cc5ceb3875bb20c2890005a4e226a4651264a5c75edb2421b52861a0a0cb50" +dependencies = [ + "tinyvec_macros", +] + +[[package]] +name = "tinyvec_macros" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" + +[[package]] +name = "tokio" +version = "1.28.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0aa32867d44e6f2ce3385e89dceb990188b8bb0fb25b0cf576647a6f98ac5105" +dependencies = [ + "autocfg", + "bytes", + "libc", + "mio", + "num_cpus", + "parking_lot", + "pin-project-lite", + "socket2", + "tokio-macros", + "windows-sys 0.48.0", +] + +[[package]] +name = "tokio-macros" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "630bdcf245f78637c13ec01ffae6187cca34625e8c63150d424b59e55af2675e" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.39", +] + +[[package]] +name = "tokio-rustls" +version = "0.23.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c43ee83903113e03984cb9e5cebe6c04a5116269e900e3ddba8f068a62adda59" +dependencies = [ + "rustls", + "tokio", + "webpki", +] + +[[package]] +name = "tokio-util" +version = "0.7.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "806fe8c2c87eccc8b3267cbae29ed3ab2d0bd37fca70ab622e46aaa9375ddb7d" +dependencies = [ + "bytes", + "futures-core", + "futures-sink", + "pin-project-lite", + "tokio", + "tracing", +] + +[[package]] +name = "tonic-build" +version = "0.8.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5bf5e9b9c0f7e0a7c027dcfaba7b2c60816c7049171f679d99ee2ff65d0de8c4" +dependencies = [ + "prettyplease", + "proc-macro2", + "prost-build", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "tower-service" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6bc1c9ce2b5135ac7f93c72918fc37feb872bdc6a5533a8b85eb4b86bfdae52" + +[[package]] +name = "tracing" +version = "0.1.37" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8ce8c33a8d48bd45d624a6e523445fd21ec13d3653cd51f681abf67418f54eb8" +dependencies = [ + "cfg-if", + "pin-project-lite", + "tracing-attributes", + "tracing-core", +] + +[[package]] +name = "tracing-attributes" +version = "0.1.24" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0f57e3ca2a01450b1a921183a9c9cbfda207fd822cef4ccb00a65402cbba7a74" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.39", +] + +[[package]] +name = "tracing-core" +version = "0.1.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0955b8137a1df6f1a2e9a37d8a6656291ff0297c1a97c24e0d8425fe2312f79a" +dependencies = [ + "once_cell", +] + +[[package]] +name = "try-lock" +version = "0.2.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3528ecfd12c466c6f163363caf2d02a71161dd5e1cc6ae7b34207ea2d42d81ed" + +[[package]] +name = "twox-hash" +version = "1.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675" +dependencies = [ + "cfg-if", + "static_assertions", +] + +[[package]] +name = "typed-builder" +version = "0.16.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34085c17941e36627a879208083e25d357243812c30e7d7387c3b954f30ade16" +dependencies = [ + "typed-builder-macro", +] + +[[package]] +name = "typed-builder-macro" +version = "0.16.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f03ca4cb38206e2bef0700092660bb74d696f808514dae47fa1467cbfe26e96e" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.39", +] + +[[package]] +name = "typenum" +version = "1.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "497961ef93d974e23eb6f433eb5fe1b7930b659f06d12dec6fc44a8f554c0bba" + +[[package]] +name = "unicode-bidi" +version = "0.3.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "92888ba5573ff080736b3648696b70cafad7d250551175acbaa4e0385b3e1460" + +[[package]] +name = "unicode-ident" +version = "1.0.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5464a87b239f13a63a501f2701565754bae92d243d4bb7eb12f6d57d2269bf4" + +[[package]] +name = "unicode-normalization" +version = "0.1.22" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c5713f0fc4b5db668a2ac63cdb7bb4469d8c9fed047b1d0292cc7b0ce2ba921" +dependencies = [ + "tinyvec", +] + +[[package]] +name = "unicode-segmentation" +version = "1.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1dd624098567895118886609431a7c3b8f516e41d30e0643f03d94592a147e36" + +[[package]] +name = "unicode-width" +version = "0.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c0edd1e5b14653f783770bce4a4dabb4a5108a5370a5f5d8cfe8710c361f6c8b" + +[[package]] +name = "unindent" +version = "0.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c7de7d73e1754487cb58364ee906a499937a0dfabd86bcb980fa99ec8c8fa2ce" + +[[package]] +name = "untrusted" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a156c684c91ea7d62626509bce3cb4e1d9ed5c4d978f7b4352658f96a4c26b4a" + +[[package]] +name = "url" +version = "2.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0d68c799ae75762b8c3fe375feb6600ef5602c883c5d21eb51c09f22b83c4643" +dependencies = [ + "form_urlencoded", + "idna", + "percent-encoding", +] + +[[package]] +name = "uuid" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "88ad59a7560b41a70d191093a945f0b87bc1deeda46fb237479708a1d6b6cdfc" +dependencies = [ + "getrandom", + "serde", +] + +[[package]] +name = "version_check" +version = "0.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" + +[[package]] +name = "walkdir" +version = "2.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "36df944cda56c7d8d8b7496af378e6b16de9284591917d307c9b4d313c44e698" +dependencies = [ + "same-file", + "winapi-util", +] + +[[package]] +name = "want" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ce8a968cb1cd110d136ff8b819a556d6fb6d919363c61534f6860c7eb172ba0" +dependencies = [ + "log", + "try-lock", +] + +[[package]] +name = "wasi" +version = "0.11.0+wasi-snapshot-preview1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" + +[[package]] +name = "wasm-bindgen" +version = "0.2.85" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b6cb788c4e39112fbe1822277ef6fb3c55cd86b95cb3d3c4c1c9597e4ac74b4" +dependencies = [ + "cfg-if", + "wasm-bindgen-macro", +] + +[[package]] +name = "wasm-bindgen-backend" +version = "0.2.85" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "35e522ed4105a9d626d885b35d62501b30d9666283a5c8be12c14a8bdafe7822" +dependencies = [ + "bumpalo", + "log", + "once_cell", + "proc-macro2", + "quote", + "syn 2.0.39", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-futures" +version = "0.4.35" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "083abe15c5d88556b77bdf7aef403625be9e327ad37c62c4e4129af740168163" +dependencies = [ + "cfg-if", + "js-sys", + "wasm-bindgen", + "web-sys", +] + +[[package]] +name = "wasm-bindgen-macro" +version = "0.2.85" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "358a79a0cb89d21db8120cbfb91392335913e4890665b1a7981d9e956903b434" +dependencies = [ + "quote", + "wasm-bindgen-macro-support", +] + +[[package]] +name = "wasm-bindgen-macro-support" +version = "0.2.85" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4783ce29f09b9d93134d41297aded3a712b7b979e9c6f28c32cb88c973a94869" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.39", + "wasm-bindgen-backend", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-shared" +version = "0.2.85" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a901d592cafaa4d711bc324edfaff879ac700b19c3dfd60058d2b445be2691eb" + +[[package]] +name = "wasm-streams" +version = "0.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6bbae3363c08332cadccd13b67db371814cd214c2524020932f0804b8cf7c078" +dependencies = [ + "futures-util", + "js-sys", + "wasm-bindgen", + "wasm-bindgen-futures", + "web-sys", +] + +[[package]] +name = "web-sys" +version = "0.3.62" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "16b5f940c7edfdc6d12126d98c9ef4d1b3d470011c47c76a6581df47ad9ba721" +dependencies = [ + "js-sys", + "wasm-bindgen", +] + +[[package]] +name = "webpki" +version = "0.22.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f095d78192e208183081cc07bc5515ef55216397af48b873e5edcd72637fa1bd" +dependencies = [ + "ring", + "untrusted", +] + +[[package]] +name = "webpki-roots" +version = "0.22.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6c71e40d7d2c34a5106301fb632274ca37242cd0c9d3e64dbece371a40a2d87" +dependencies = [ + "webpki", +] + +[[package]] +name = "which" +version = "4.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2441c784c52b289a054b7201fc93253e288f094e2f4be9058343127c4226a269" +dependencies = [ + "either", + "libc", + "once_cell", +] + +[[package]] +name = "winapi" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" +dependencies = [ + "winapi-i686-pc-windows-gnu", + "winapi-x86_64-pc-windows-gnu", +] + +[[package]] +name = "winapi-i686-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" + +[[package]] +name = "winapi-util" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "70ec6ce85bb158151cae5e5c87f95a8e97d2c0c4b001223f33a334e3ce5de178" +dependencies = [ + "winapi", +] + +[[package]] +name = "winapi-x86_64-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" + +[[package]] +name = "windows" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e686886bc078bc1b0b600cac0147aadb815089b6e4da64016cbd754b6342700f" +dependencies = [ + "windows-targets 0.48.0", +] + +[[package]] +name = "windows-sys" +version = "0.45.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "75283be5efb2831d37ea142365f009c02ec203cd29a3ebecbc093d52315b66d0" +dependencies = [ + "windows-targets 0.42.2", +] + +[[package]] +name = "windows-sys" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "677d2418bec65e3338edb076e806bc1ec15693c5d0104683f2efe857f61056a9" +dependencies = [ + "windows-targets 0.48.0", +] + +[[package]] +name = "windows-targets" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e5180c00cd44c9b1c88adb3693291f1cd93605ded80c250a75d472756b4d071" +dependencies = [ + "windows_aarch64_gnullvm 0.42.2", + "windows_aarch64_msvc 0.42.2", + "windows_i686_gnu 0.42.2", + "windows_i686_msvc 0.42.2", + "windows_x86_64_gnu 0.42.2", + "windows_x86_64_gnullvm 0.42.2", + "windows_x86_64_msvc 0.42.2", +] + +[[package]] +name = "windows-targets" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7b1eb6f0cd7c80c79759c929114ef071b87354ce476d9d94271031c0497adfd5" +dependencies = [ + "windows_aarch64_gnullvm 0.48.0", + "windows_aarch64_msvc 0.48.0", + "windows_i686_gnu 0.48.0", + "windows_i686_msvc 0.48.0", + "windows_x86_64_gnu 0.48.0", + "windows_x86_64_gnullvm 0.48.0", + "windows_x86_64_msvc 0.48.0", +] + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "597a5118570b68bc08d8d59125332c54f1ba9d9adeedeef5b99b02ba2b0698f8" + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91ae572e1b79dba883e0d315474df7305d12f569b400fcf90581b06062f7e1bc" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e08e8864a60f06ef0d0ff4ba04124db8b0fb3be5776a5cd47641e942e58c4d43" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b2ef27e0d7bdfcfc7b868b317c1d32c641a6fe4629c171b8928c7b08d98d7cf3" + +[[package]] +name = "windows_i686_gnu" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c61d927d8da41da96a81f029489353e68739737d3beca43145c8afec9a31a84f" + +[[package]] +name = "windows_i686_gnu" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "622a1962a7db830d6fd0a69683c80a18fda201879f0f447f065a3b7467daa241" + +[[package]] +name = "windows_i686_msvc" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "44d840b6ec649f480a41c8d80f9c65108b92d89345dd94027bfe06ac444d1060" + +[[package]] +name = "windows_i686_msvc" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4542c6e364ce21bf45d69fdd2a8e455fa38d316158cfd43b3ac1c5b1b19f8e00" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8de912b8b8feb55c064867cf047dda097f92d51efad5b491dfb98f6bbb70cb36" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ca2b8a661f7628cbd23440e50b05d705db3686f894fc9580820623656af974b1" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "26d41b46a36d453748aedef1486d5c7a85db22e56aff34643984ea85514e94a3" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7896dbc1f41e08872e9d5e8f8baa8fdd2677f29468c4e156210174edc7f7b953" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9aec5da331524158c6d1a4ac0ab1541149c0b9505fde06423b02f5ef0106b9f0" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1a515f5799fe4961cb532f983ce2b23082366b898e52ffbce459c86f67c8378a" + +[[package]] +name = "winreg" +version = "0.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "80d0f4e272c85def139476380b12f9ac60926689dd2e01d4923222f40580869d" +dependencies = [ + "winapi", +] + +[[package]] +name = "xz2" +version = "0.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "388c44dc09d76f1536602ead6d325eb532f5c122f17782bd57fb47baeeb767e2" +dependencies = [ + "lzma-sys", +] + +[[package]] +name = "zstd" +version = "0.12.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1a27595e173641171fc74a1232b7b1c7a7cb6e18222c11e9dfb9888fa424c53c" +dependencies = [ + "zstd-safe 6.0.5+zstd.1.5.4", +] + +[[package]] +name = "zstd" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bffb3309596d527cfcba7dfc6ed6052f1d39dfbd7c867aa2e865e4a449c10110" +dependencies = [ + "zstd-safe 7.0.0", +] + +[[package]] +name = "zstd-safe" +version = "6.0.5+zstd.1.5.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d56d9e60b4b1758206c238a10165fbcae3ca37b01744e394c463463f6529d23b" +dependencies = [ + "libc", + "zstd-sys", +] + +[[package]] +name = "zstd-safe" +version = "7.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43747c7422e2924c11144d5229878b98180ef8b06cca4ab5af37afc8a8d8ea3e" +dependencies = [ + "zstd-sys", +] + +[[package]] +name = "zstd-sys" +version = "2.0.8+zstd.1.5.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5556e6ee25d32df2586c098bbfa278803692a20d0ab9565e049480d52707ec8c" +dependencies = [ + "cc", + "libc", + "pkg-config", +] diff --git a/Cargo.toml b/Cargo.toml new file mode 100644 index 0000000..b553351 --- /dev/null +++ b/Cargo.toml @@ -0,0 +1,57 @@ +# 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. + +[package] +name = "raysql" +description = "RaySQL: DataFusion on Ray" +homepage = "https://github.com/datafusion-contrib/ray-sql" +repository = "https://github.com/datafusion-contrib/ray-sql" +authors = ["Andy Grove ", "Frank Luan "] +version = "0.6.0" +edition = "2021" +readme = "README.md" +license = "Apache-2.0" +rust-version = "1.62" +build = "build.rs" + +[dependencies] +datafusion = { version = "33.0.0", features = ["pyarrow", "avro"] } +datafusion-proto = "33.0.0" +datafusion-python = "33.0.0" +futures = "0.3" +glob = "0.3" +log = "0.4" +prost = "0.12" +prost-types = "0.12" +pyo3 = { version = "0.20", features = ["extension-module", "abi3", "abi3-py37"] } +tokio = { version = "1.24", features = ["macros", "rt", "rt-multi-thread", "sync"] } +uuid = "1.2" + +[build-dependencies] +rustc_version = "0.4.0" +tonic-build = { version = "0.8", default-features = false, features = ["transport", "prost"] } + +[lib] +name = "raysql" +crate-type = ["cdylib", "rlib"] + +[package.metadata.maturin] +name = "raysql._raysql_internal" + +[profile.release] +codegen-units = 1 +lto = true \ No newline at end of file diff --git a/LICENSE b/LICENSE new file mode 100644 index 0000000..261eeb9 --- /dev/null +++ b/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/NOTICE b/NOTICE new file mode 100644 index 0000000..1e8e163 --- /dev/null +++ b/NOTICE @@ -0,0 +1,2 @@ +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/README.md b/README.md index f3cc573..ebd7803 100644 --- a/README.md +++ b/README.md @@ -1 +1,149 @@ -# datafusion-ray + + +# datafusion-ray: DataFusion on Ray + +This is a research project to evaluate performing distributed SQL queries from Python, using +[Ray](https://www.ray.io/) and [DataFusion](https://github.com/apache/arrow-datafusion). + +## Goals + +- Demonstrate how easily new systems can be built on top of DataFusion. See the [design documentation](./docs/README.md) + to understand how RaySQL works. +- Drive requirements for DataFusion's [Python bindings](https://github.com/apache/arrow-datafusion-python). +- Create content for an interesting blog post or conference talk. + +## Non Goals + +- Build and support a production system. + +## Example + +Run the following example live in your browser using a Google Colab [notebook](https://colab.research.google.com/drive/1tmSX0Lu6UFh58_-DBUVoyYx6BoXHOszP?usp=sharing). + +```python +import os +import pandas as pd +import ray + +from raysql import RaySqlContext + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + +# Start a local cluster +ray.init(resources={"worker": 1}) + +# Create a context and register a table +ctx = RaySqlContext(2, use_ray_shuffle=True) +# Register either a CSV or Parquet file +# ctx.register_csv("tips", f"{SCRIPT_DIR}/tips.csv", True) +ctx.register_parquet("tips", f"{SCRIPT_DIR}/tips.parquet") + +result_set = ctx.sql( + "select sex, smoker, avg(tip/total_bill) as tip_pct from tips group by sex, smoker" +) +for record_batch in result_set: + print(record_batch.to_pandas()) +``` + +## Status + +- RaySQL can run all queries in the TPC-H benchmark + +## Features + +- Mature SQL support (CTEs, joins, subqueries, etc) thanks to DataFusion +- Support for CSV and Parquet files + +## Limitations + +- Requires a shared file system currently + +## Performance + +This chart shows the performance of RaySQL compared to Apache Spark for +[SQLBench-H](https://sqlbenchmarks.io/sqlbench-h/) at a very small data set (10GB), running on a desktop (Threadripper +with 24 physical cores). Both RaySQL and Spark are configured with 24 executors. + +### Overall Time + +RaySQL is ~1.9x faster overall for this scale factor and environment with disk-based shuffle. + +![SQLBench-H Total](./docs/sqlbench-h-total.png) + +### Per Query Time + +Spark is much faster on some queries, likely due to broadcast exchanges, which RaySQL hasn't implemented yet. + +![SQLBench-H Per Query](./docs/sqlbench-h-per-query.png) + +### Performance Plan + +I'm planning on experimenting with the following changes to improve performance: + +- Make better use of Ray futures to run more tasks in parallel +- Use Ray object store for shuffle data transfer to reduce disk I/O cost +- Keep upgrading to newer versions of DataFusion to pick up the latest optimizations + +## Building + +```bash +# prepare development environment (used to build wheel / install in development) +python3 -m venv venv +# activate the venv +source venv/bin/activate +# update pip itself if necessary +python -m pip install -U pip +# install dependencies (for Python 3.8+) +python -m pip install -r requirements-in.txt +``` + +Whenever rust code changes (your changes or via `git pull`): + +```bash +# make sure you activate the venv using "source venv/bin/activate" first +maturin develop +python -m pytest +``` + +## Benchmarking + +Create a release build when running benchmarks, then use pip to install the wheel. + +```bash +maturin develop --release +``` + +## How to update dependencies + +To change test dependencies, change the `requirements.in` and run + +```bash +# install pip-tools (this can be done only once), also consider running in venv +python -m pip install pip-tools +python -m piptools compile --generate-hashes -o requirements-310.txt +``` + +To update dependencies, run with `-U` + +```bash +python -m piptools compile -U --generate-hashes -o requirements-310.txt +``` + +More details [here](https://github.com/jazzband/pip-tools) diff --git a/build.rs b/build.rs new file mode 100644 index 0000000..51b30d0 --- /dev/null +++ b/build.rs @@ -0,0 +1,54 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::path::Path; + +fn main() -> Result<(), String> { + use std::io::Write; + + let out = std::path::PathBuf::from(std::env::var("OUT_DIR").unwrap()); + + // for use in docker build where file changes can be wonky + println!("cargo:rerun-if-env-changed=FORCE_REBUILD"); + + let version = rustc_version::version().unwrap(); + println!("cargo:rustc-env=RUSTC_VERSION={version}"); + + let path = "src/proto/generated/protobuf.rs"; + + // We don't include the proto files in releases so that downstreams + // do not need to have PROTOC included + if Path::new("src/proto/raysql.proto").exists() { + println!("cargo:rerun-if-changed=src/proto/datafusion.proto"); + println!("cargo:rerun-if-changed=src/proto/raysql.proto"); + tonic_build::configure() + .extern_path(".datafusion", "::datafusion_proto::protobuf") + .compile(&["src/proto/raysql.proto"], &["src/proto"]) + .map_err(|e| format!("protobuf compilation failed: {e}"))?; + let generated_source_path = out.join("raysql.protobuf.rs"); + let code = std::fs::read_to_string(generated_source_path).unwrap(); + let mut file = std::fs::OpenOptions::new() + .write(true) + .truncate(true) + .create(true) + .open(path) + .unwrap(); + file.write_all(code.as_str().as_ref()).unwrap(); + } + + Ok(()) +} diff --git a/docs/README.md b/docs/README.md new file mode 100644 index 0000000..516c338 --- /dev/null +++ b/docs/README.md @@ -0,0 +1,279 @@ + + +# RaySQL Design Documentation + +RaySQL is a distributed SQL query engine that is powered by DataFusion. + +DataFusion provides a high-performance query engine that is already partition-aware, with partitions being executed +in parallel in separate threads. RaySQL provides a distributed query planner that translates a DataFusion physical +plan into a distributed plan. + +Let's walk through an example to see how that works. We'll use [SQLBench-H](https://github.com/sql-benchmarks/sqlbench-h) +query 3 for the example. This is an aggregate query with a three-way join. + +_SQLBench-H Query 3_ + +```sql +-- SQLBench-H query 3 derived from TPC-H query 3 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + l_orderkey, + sum(l_extendedprice * (1 - l_discount)) as revenue, + o_orderdate, + o_shippriority +from + customer, + orders, + lineitem +where + c_mktsegment = 'HOUSEHOLD' + and c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate < date '1995-03-21' + and l_shipdate > date '1995-03-21' +group by + l_orderkey, + o_orderdate, + o_shippriority +order by + revenue desc, + o_orderdate limit 10; +``` + +## DataFusion's Logical Plan + +DataFusion produces the following optimized _logical_ query plan. Note that this plan does not have the +concept of partitions yet. + +```text +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)]] + Inner Join: orders.o_orderkey = lineitem.l_orderkey + Inner Join: customer.c_custkey = orders.o_custkey + Filter: customer.c_mktsegment = Utf8("BUILDING") + TableScan: customer projection=[c_custkey, c_mktsegment], 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: lineitem.l_shipdate > Date32("9204") + TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate > Date32("9204")] +``` + +## DataFusion's Physical Plan + +DataFusion's physical plan lists all the files to be queried, and they are organized into partitions to allow for +parallel execution within a single process. In this example, the level of concurrency was configured to be four, so +we see `partitions={4 groups: [[ ... ]]` in the leaf `ParquetExec` nodes, with the filenames listed in four groups. + +_DataFusion will soon support parallel execution for single Parquet files but for now the parallelism is based on +splitting the available files into separate groups, so RaySQL will not yet scale well for single-file inputs._ + +Here is the full physical plan for query 3. + +```text +GlobalLimitExec: skip=0, fetch=10 + SortPreservingMergeExec: [revenue@1 DESC,o_orderdate@2 ASC NULLS LAST] + SortExec: [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 + 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@6 as l_orderkey, o_orderdate@4 as o_orderdate, o_shippriority@5 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: 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 + 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 + 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 = Utf8("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 < Date32("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 + 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 > Date32("9204"), pruning_predicate=l_shipdate_max@0 > 9204, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate] +``` + +## Partitioning & Distribution + +The partitioning scheme changes throughout the plan and this is the most important concept to +understand in order to understand RaySQL's design. Changes in partitioning are implemented by the `RepartitionExec` +operator in DataFusion and are happen in the following scenarios. + +### Joins + +The first join to happen is between `customer` and `orders`. The join condition is +`customers.c_custkey = orders.o_custkey`. To perform this join in parallel we first +need to repartition the data on the join keys so that the data for customers with `c_custkey = 1` +and the data for orders with `o_custkey = 1` can be sent to the same thread or node for +processing. This allows the join to happen in parallel. This is known as a hash-partitioned +join and is implemented by the `HashJoinExec` operator in DataFusion. + +We can see that `DataFusion` has inserted `RepartitionExec` operators around both inputs to the join. + +### Aggregates + +There are multiple approaches to distributed aggregate queries. Here are two popular approaches: + +- Perform aggregates in parallel on each partition, where the resulting data for each partition could contain + duplicate grouping keys and then perform a final aggregate on the intermediate aggregates to remove the duplicates. +- Partition the input data by the grouping keys so that the aggregates from each partition can simply be merged to + produce the final result. + +For this example query, the data is already partitioned on the aggregate's grouping key so the latter approach is used. + +### Sort + +Sort also has multiple approaches. + +- The input partitions can be collapsed down to a single partition and then sorted +- Partitions can be sorted in parallel and then merged using a sort-preserving merge + +DataFusion and RaySQL currently the first approach, but there is a DataFusion PR open for implementing the second. + +### Limit + +- The input partitions can be collapsed down to a single partition and then have the limit applied +- The limit can be pushed down to each partition as well + +## Query Stages + +The first two query stages to be executed will read the `customer` and `order` parquet files and reparition them by the join keys `c_custkey` and `o_custkey`. + +```text +Query Stage #0: +ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) + 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 = Utf8("BUILDING"), pruning_predicate=c_mktsegment_min@0 <= BUILDING AND BUILDING <= c_mktsegment_max@1, projection=[c_custkey, c_mktsegment] + +Query Stage #1: +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 < Date32("9204"), pruning_predicate=o_orderdate_min@0 < 9204, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority] +``` + +Now that these inputs are partitioned by join key, we can execute the join itself. The two inputs to the join are instances of `ShuffleReaderExec` that +read the shuffle files produced by the previous stages. The result of the join is repartitioned by `o_order_key` in preparation for the next join. + +```text +Query Stage #2: +ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_orderkey", index: 2 }], 4)) + 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_partitions=4) + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=1, input_partitions=4) +``` + +We continue to prepare for the next join with `lineitem` by repartitioning the parquet files by the join key. + +```text +Query Stage #3: +ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) + 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 > Date32("9204"), pruning_predicate=l_shipdate_max@0 > 9204, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate] +``` + +With the inputs repartitioned by the join keys, we can execute a query stage to perform the join. However, in this case, we also perform an aggregate in the +same stage without any additional repartitioning. This is possible because the aggregate grouping includes `l_orderkey` which we already partitioned on. This +means that we can perform this aggregate in parallel and guarantee that there will be no duplicates in the output of each aggregate. Pretty neat! + +The output of this shuffle is partitioned by `l_orderkey`, `o_orderdate`, and `o_shippriority` in preparation for the `ORDER BY` part of the query. + +```text +Query Stage #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@6 as l_orderkey, o_orderdate@4 as o_orderdate, o_shippriority@5 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: 2 }, Column { name: "l_orderkey", index: 0 })] + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=2, input_partitions=4) + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=3, input_partitions=4) +``` + +Now we perform a final aggregate (which is maybe redundant?) and then sort the results in parallel across the +partitions. + +```text +Query Stage #5: +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: [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_partitions=4) +``` + +Finally, we have a query stage that reads the sorted results and merges them into a single partition (preserving the +sort order) and applies the `LIMIT` clause. + +```text +Query Stage #6: +GlobalLimitExec: skip=0, fetch=10 + SortPreservingMergeExec: [revenue@1 DESC,o_orderdate@2 ASC NULLS LAST] + ShuffleReaderExec(stage_id=5, input_partitions=4) +``` + +## Distributed Scheduling + +In the previous section we walked through the plan as if we were executing one query stage at a time. However, some +of these query stages could execute in parallel (assuming enough resource is available). + +For example, we could execute query stage 3 to repartition the `lineitem` input without waiting for query stages 0, 1, +and 2 to finish. + +More generally, we can typically execute all remaining leaf nodes of the plan concurrently. + +The `execute_query_stage` method in `context.py` is a remote method that recursively walks the query plan and executes +child plans, building up a DAG of futures. + +## Distributed Shuffle + +The output of each query stage needs to be persisted somewhere so that the next query stage can read it. Currently, +RaySQL is just writing the output to disk in Arrow IPC format, and this means that RaySQL is not truly distributed +yet because it requires a shared file system. It would be better to use the Ray object store instead, as +proposed [here](https://github.com/datafusion-contrib/ray-sql/issues/22). + +DataFusion's `RepartitionExec` uses threads and channels within a single process and is not suitable for a +distributed query engine, so RaySQL rewrites the physical plan and replaces the `RepartionExec` with a pair of +operators to perform a "shuffle". These are the `ShuffleWriterExec` and `ShuffleReaderExec`. + +### Shuffle Writes + +`ShuffleWriterExec` reads input partitions and repartitions them, using the same `BatchPartitioner` that DataFusion +uses, then writes the output to disk in Arrow IPC format. + +### Shuffle Reads + +`ShuffleReaderExec` reads the shuffle files written by the `ShuffleWriterExec`. diff --git a/docs/sqlbench-h-per-query.png b/docs/sqlbench-h-per-query.png new file mode 100644 index 0000000000000000000000000000000000000000..86757688832a02233d58138a0a369a089ffd57b1 GIT binary patch literal 20772 zcmdVCc{JAR+dh12PNftnL<40Al_`XhA;~-wl4Pcgk!e>{DoQd$$UKBH&zhu2GG|Vi z#|HEG9#{L>`}_R{Na`}X^s1)o?bKG z8!RXceV23$YbxGUoV=LMA7w?;&)nycZguhO#y5vn;a(|T--bqL$PXOu>&Q}2&isG$ z(*C4_s%P|D`JKlC7J+)X*2!yG-9NW<6g>T25x^}fs`ui>i^5CY*Nz-HqLFZB=j`lk z^5yUEs5fnTR3yeHMxl7W{`6~KZbO5HjEsy$bGiXJR!vQW}?zAA0MA&=h4=L3iruIZNIyB8$u78Xyn;; z9GegBn5sEzV9@*hdo0eC*xKCKQH=00*nZT{EoQqhq$`Y%hb)&6}w%(Ue?8GwF3LziI4m>@4LFX8ukka zXiRrr%_v>?rBEFqqHa>*uX&GA(AIL^tR}MYwaL_NT{K>*-%;cuoMu$$m@H(|8aFxIOdct>*Q)7R zEqV6w@r;`+Zl`eVoU3QgpO2APUU0DKDAbIMj2yx)md>@L8;I2pyopx`ndrPa{>K3E zNH%M|-s|h>J-%2InP%CTV)5m<@vn)Ak2z@p0Rg_7S<}P#%`CfL`?!r2^q=W!Sk|<( zOzHGm_JDC)EX@3Lo9=LNu$iA9FS8nMO0)bN{TDuC+vYq!H7aOYN$0V=IPJWT7JsV8 zw)OP)C+<1v8*1o6EUQ1tE8Sy$%A&XO5w<%dd4C2wg`#p+myzyMgvdU5OPIX>a1m?> z*7!w{N?u8+X>M}3cXTvOj@3h;!b1KLw{}yKYV5=(v2pV^zMH5wY*6>1rhhh4=$Z9IMrrX=sug-@yg)h#RRfu0|Vyl`}bGEw3n8a;2+v|w)0qzdM?eHdHk*xW)yeJ z{uCvdK@9ouVO8vdWz(~;v7&JaS6A1T#*PAqL?OG*XZ4B7Rj@`XDk|(@c#3A~#b(~C zj=eBJm9u9fGBYzZaP7&_w&sT1-8{OP(!R!Dq@=X8f`zQVM8AK3%Aoj)s%DDD3!ir) z4t)bK;^wZUDdCd&v7**8uQEnK%b@MrjRyNtXB{N}kc5coRJE>QE<+6V)Tfv9n z97IwT4IRb8tgCb{6etz}t?(TxfqTTYQl|i+;T-x`% zIYhT>myKy2yMg3x#Z#y5CnhGQXe6K4x830@F+akecD6uG(5gved1j`Ela+xX`B!fgJ|M5QNc z7cRujO$_c+R#qn0of>I@)85m|Hh&5eOR}g>(96H{)OqB~9&K&y#%H?f2si50j|H86 ze0{~#^kZsDt0q$1ZpYTR;r{&mf`Z;Fyiy2Ral;4n73#)1iV&loe*XNK`J|K(A0MCi zkC!Vg>l3Os4VzVs{#nT&9nqLA*rHux&gL*wzkByIVM}y$baD2*mA!MJ`T0lX)k@qZ zZSxoJ-`|p^mvg$pe^-^ZB!aEY?~YAxR1kVb+puo)sT#>@xw&OabE#9~T^WWYZpnx( zxxJ%pdCd*U>N#$cmN{0u-dVzU^iuTMvuA@M zBAzv-YIE=2eca^T-4`A!I}uMjX1Z_cmAK`wc+ME>mn_atTfMzSOT090xMmVZ=Xfo9 zJ`r^qO6Z^0x4<28>E}LU6t+D$=yqJxao|4je{ACEK$s<)k&zMck`%*|LYqi0Jkl9s zjxye}`O%rCRee`z6sFcS6gc#=8EmExq+@1|$atLhi^Ow|(+4+EKjPA?Q&$#`N9-Ab z@exap8(wSi>eVaDy13hn0_Nw0d3unJ0v*-s`7<|9WwJUHZRb5}uGiPy{ZztZF0@pn zv*csMVy z*;2~0X9tlzTE$s&EF0@$10=YRc#8Dk{kivl^ykh`k89i8|59A*^W}?fhp4HSo)J6ZaGqoNY_yd@JRIqpG%9u1C5eBtgzj{6t*p4I3I7{|BK7YK6 z`3OHhyFr=SeYX1b*=nC+xe*J6c&a336qS`j*kt77h6ie6ooz&&N6fX~G&c*^{9Z9N z){$Bj!q;CY(FTKZ9BUWRKQ7`t5}IyUlA@KS7uE{X(+u=bIVCU8`Pr^5E78pKX6A_M z&4>gW;^qk9{5-6pj!8BXY;9>IJyHC&-t3sIY;M+uEoAOG8i`cKBy8J|Ytxo}?$rwM zpKrJ2I@e8JA7%2nIkNKC0?U=vt5=7mYt)x|TotsalaBJ7OAbD$8v}PeRk*Y%`_7#^ zvUhhp7@k^k+v3IldoXMBnB7#LiE#bc`v?0HtmcKSnm+P!z*}Ts$2Cq&2b3d=lcVo1 z-gBQCA0KJS${k(SbiOkCyI>^)pP@#%AHAJUzt$Buw^rGWOI%l7o%1U_HzD123NWf8 zKw}L)P5o}kGE<~F_|NkMt|*eR#RnTRj3jqSdY-*}`EsFsuU~OV2}viTEm?9%)fy@) zDv@nR-DjS&wHR#ZIt^nC%*)H`kk3V;J*}jqb^N&Zpr6d_0}pST-idIkm21pjdQ%14 z@bT&D>IN`Yvvug}owxRj|J0 zDes{}Dq_wfpZ{99mIT70TwE}d$>~Gs&bGAogZ~wboEu{iMJ-71eQ{nUax`;%DtvL1 z9YPYIRSaTDzw6iuB>7zHmc82A&CH2;I(xXe6$S?fE8o9=t|1n75hHUjb83-lVbk z{_b9DmYK4@4T*j5LCy5DEke~vXAA6yzPyMfrl~A3E8SlcB`iLM@Gv(~uk7^mn@opO zvjL4*nK4ktX=PGe)3USJHAOYn z_u92*f{PFC@WiB|oeo{)e8C zWOFZKIzr-rzt~uQG9zjx=jq<}f1~kWJ|kQG?XNwW7C?Oj~7PoMUKdnI7CfP{wd@lTg6+y4n$ z3N@-pMrEauSI0zbI*WC0Zf0$&kz9T@G2w_jbsC{mOIrE{^53~aw)pVyaI!lm-+Fl?jw>Zf+JjuabUNWO3MN8MR~}mqyYh1#L0EJ9p}l zxkrKx3LQV#>Z16v$tiRhPX{`&z%K-%!A9f1em&<$CZQD$W zXv0xkrD$iQW2cf(Q{>J|E;RnLfDEzsm7+Uwv!%;RS35kWI|}^wzMRG8r6Z`zuUt!` zQ5DSd+xqIv5922y4zjyM9XX;qkrlr>7c5SfEiYzdi9L9T>s$QptM&^B&`#A#v$b;D z%*=Q6sE&4qVIvR)9RtGwVPRq5L*ih_k81Aj#aM)i#py09gbgJ4%qP=0r}yjB&;A^z z>hQGL?z8sTJ*B5lpOO+(N9POA6Aiy>mq)%FCiUy#!-q+<)7MT`WB0jnBgv*Mmw=gv zZtHLsebuDv*Nk_xW}4n1`=k)U`=O#jPFZ}C&ZUs&Eh_q;prE)^?F_(QHRpaVq`&9^ zj)oJs0gAVSjx+0O{6HyR`}Z5`|3_YK7iEDon9Z40N$8y0qZv|f+n#$jaiVh3(319R z-;DpG-LjSJ?Bdh;NBo%!lczd22dITTjosDGO*!s)D25XKUd(>~?*yr9Yz9gXomp#& zlXGFMnpdvmgT07Bh!`$sFw(HJw3KkO90xS2bEpw_dE|1Gchbz zM_w?mtOD$5jFsE`=iRb}Oct+&%LXNGjSaSaAH(!AFWeq#&M?|ibe|8^8(||xyV?sJ zCO;gA>e+eCw)~!F4d9nbl!OO36$1j=5wS>BXn3W2b> zu!q2lpCUN5n}@qHjxGFnL%n`|ErJs<7sR5$d%~UfZ{EB~NQjb&It3C_iQ@ssCL;kp z7PJbcVL6s0eswBIKhKuH@3hWj^~8omWmd%}qM8WxSv<-L3iUVEZ)+&?@{(5rZS*KI zGCj+zCIv)LuJg{FI~fn3zp-W$BOB-hgz5KSKEOI?A#{#RrQsUn0r2OZ=lwORK-$ru zm8`I!h5eCN39<0=d&TgV7bcY-KQ>_T1nqlHmHevYK8LBwxV!l<*A5hv8ooVkmrvtQn=+lN8m;>R6?v_RUAGj#tcBOi#x zBNtc(d$Z18z##^-eCIY+`-te-W3JeJeS+jA+Zh4@9X6&RUwhgsjefpE4q9RiB zAbgw(bRKKxMDc^o`kVC!ZPU?8M)-=zFyxu zjFJS@`l&``o^wbbL13-q^@^O+kmbx6w#G3nN9NhIC7`%{JV)~UB8(OE)a4(BE4_bj zp@W%#+^Tiy@<{yMUo-rof<--6_TI!uR?QG#e6A37JP0=l%8o20Nv46>=gXKsI(aL#=*h4@i_2G38P&D(-AOYFbrWV{ey$ae)Qa2oSYw2i)biHa&XbJ0zh}; zrSs{qjT}T?gZxV;5SX*lmVYvjYw32Kof50PYA_ zH61J{DjK%)nh&>G#2XU>u>mRA&Y_m9#Bblfo9^&sO9lj*1APG`{ROo=5|r(1(b)iAY; zBa@_js863c(*OW933AB_mH%ANL*yh3s%D*_0Yrkf>eY;24t(ip6ZCpC_#sj08Sb7FzHc5ku@3JM01HW~sXmpBIJZoL8yO#s2ce&E3H9lp}%IUb-1 zC%)a`Ywa61o3|>8vHte<_FUQWyrljqv8z+h2#+v0M}L>l1}H`q0s(dDcI-$(IP5(- zdaCf}xa-K57p=vvSqW#t&NCfu&9->Bv^efH%(^@y7peY|5}ovY8x6(&$2%rFaH_|_ zu&AL*Zj4u8aCLJVLaxZpS-)XJy*JfL$BJDp1h^1@+8BLeCHPofRNxw*SQ_AQU{k3*gCZC^>Ot zOpMq{k#k@}n!a#jvbvmpu66A;l6lo+qb8~Qla^nK>%AA|8JvVL?qN61tduUbvIt?q&nXH z?$!$Ly&1NG!Z&j(?iDnr@r4{QwJj{U6Ogg;s)GUWwZy`YllPcK>Hb2NR;RZq?xvPAa9VM zPfA4uN^KxQo;D#0ozNL^kVlkruN(H^g|dIuAAt2g^inR$!3K+cdXq&B>@)Ej#mHkV zxx7L`B($m#3Lw)|!KNj%{I)o`E;WOTnaj%u`;{V+P$WD#dRYq4TvJCU6h%U`*m$w} zKH0kjHe@@G>Jg^`0)~xM^ngwsv0aH1C}}^tE(H+V@JyGV5J-R@j)U> z5o!%!bY4kGNnBb6y`WEHLiq4k$8xtJRf0;;0g`V{+#@3^%XRG7S>RdfjT?iC=2=CO zLEaF;E~}9A$LP55fES9~CR1?oIKm0ze^NW04&k+Fe%IEfubqDOvfAc9=s8|TSv)W< zE{+Y9q5`hz*851C?GFLmTNN_e8Xp(;zM|qlh0WWFieV6vG~?WqAOSGYnqV-qil`{% zO86k4P*jFZ{tjPborWl>1A>B@)9nrcYiO905+mIzbB}>e$0MUuy^jJrZdg5_@6(gi zA3rXcIWgdVIRyMB8)AjK+|>aI@;0mw)F`kwMFDH5o^VDL)cy8@I?-r45z0pX@BIil zFCMdH%a-VwnJWc_OAl@moL2%zC-oUgH0YG1+IAE^7I8?>%d;JQ+evK6NA3m9pZ1}x z@{33}(pmgEMK_LZwMVPPSn;UxAT`Uhvz31eV!f`MD}?CkVH09nxHiBCx2f)xS? zJ4^j-=gB&2eULqe5~DoTPMx}C(NTVf0dQ30zyz@gjs-s_m31^U!2tov{=@5*2v((W~L zSxNCDwN+JDm!we}$Ac5yNcbSi0$#s!H)(|t{a~H+8C2wO zaTVsE_sMdgMTS~){;&!1-~@eu`V$eU>OY8+ctm{@4)g=fO>?wR0R4lEcg)Rv510gq z8bUGFfP{d)?iB>X(|`T-*IZw?okotOrkTr5vIul%wYb-+{-XszlRlzjeUDLp^Y2Kv ziSKWZtZvvwmS8Qr{Sa{z?_ydyyih`ciT~Tn(Vi4J(C^KJc)4+775c}(IZ1$mfTR!h zRg${;d1(ATYLXLxLC{(-Bh5t80PcQsGv~sGtsSUW(c^mBR}*DtDo@T$bFSr<6iS=k zzP<$H?6!`dGc~z(UFjsU5JVryuJDX&6IRIDS|T*Lg{ht{#N2E3ymuupW8(+)?#cFu z2L2#{;MG1)Pu@)NT3&L^C0K$$c;MyMluTTryfQn-Le!zwJv=pE`ec1m9A|N4-ZnR% z!1tNSaxHthK~+B%vU$ul`1$F{LL~nNu$!b0nVXBIdzq#wp3Co1gp^e74BOblJJ(ER zH3L{&duShVIxObq#e5kW99D?OvzRR!#IT3y}iq>g&^ZNfLW`;vlQ)4*rXGEt}JM3q8im!LRzCW(1Z<! z@`uqt&2y_KcSwD!_k=O-+YM&~eZdI=w>3=+zTMbM{(RIMr!MG^tWV7-O>vtX0=Tap zswPgwK?z>#1Hr=}D#1LrEOIdQ{oW8cEJvE>!cVzW;SG2znxZQa^W1Sz4Uj_@1UmE{ zFZ3%!$PFE@{d&JDwy;RQRF`~s_4{A@=CKRLYMbSJnYhBhL;7RC26|Z=z`e>YjPN36y1Lg!K!Cx z3l!boEe{TjAcp^3*9Twp1mul)rmNDmiqe9wXib z1qUZw9WT*FMU?Z8LmcDSxJy)nKnxJ$?D9~z40Vt>0*t3JDFkr@;Xx3Z zh9SUsglg|!lj`TV%OtD>NzG?4J&sc&y7J9$-XNI#YxAtCI*8)uO-vBn_JbIeufr0O zBJ|%omWWX)bhH3o`@*B3q}2E6BW~e1?Ri&~D`)~YqUlz$G&g)&UcMG$jSTS8o0xgkAo@g9m5C$maX)z@8Cf zMwkv{EBX13X{aqP!*}D(7%&Yj|M~2XS~=FihsqMFNS}Co@ihHBC8YQU=s;}$*|yn+ zeJ(C8k9hR8(DJ&E1wxO5O&)Ek|E&0Gz+=Q3dGH`j_xGG+lSdt?4aENMN=3O%w3tP4 zN=O(Yn>6y*9NQ;PIuU3P*g7SLXFBvNK_6t*o`2c7%EZJ3rP~3`XF94-<}`qXil~#s zBSjy|JjmfbKbjX92RRWT;>hZ>cVukVIe0c-1K~LdyAgb=z<%NDp$X|4y=Fhf*Q`dlL6`4YXK!U=PgsH6Gz ztTP+35zbibM;niVy!-CP-+I+vwb}*B)0opzhk_^_Yu>__wPT0l9Kd*aln^UH8g@q z?ZQS$z z!M}DL^|2Uy0zx0QzFO#!*~CDt8?N!MRkNycAR!}yj0A&hVep$jfV-mPiy-Add89++ zBG7}R2L;iUEHBMtlRrNNH`@?fBQbvlEQku^a6R9@n@n%UUyj$Fo4xfKyzT_@CWwk#OrgZ0 zWP*B6JVhx=nzxPN1azo)AKXk0El-kc0884{u>v`mkn=E@2@9vcp~E>>l~9o2knhBz ztskDi@&_$?$GrL#y@i!E1G(-)W8<&iapd%LTdD&3m>?D@ZHGjy5VB!kNiv2*a|sAE3{{gj#c>?(zh*-&;qYX3_LoBB5V-JuOWVNfk74|c=yAZa6_M2w zY9U7?kb+1ONweK;dMpi<6i9AcwER8;`8c6U;D%~ps16@tmu>BvEIelBd*u5_nYR-H zM$`l4B#oITN<%ySh*u&s#e!F`gp^U#q32i|C`wY)+d2s(X!i)&m5=9JjQYC_eXxx z5P6*^KNUY4$+BEMJF26OHnxc}VT} zU)}q$2Vb2P^{r^u*RdKM-L3_90xLu0RXnL`@qQ>vFaF8n(Vw|fjE?HD(2))|czOb$ zgR7$?jj+(U|3J6}W}lt>7Y86mZU9bC1O1f@a-B3Q!1kQ`2M@Tam_w|Fq)3mZAyEG8 z{5(n@J*^OggG)&(cQmJO*YfxdSFJAI8qlY_3&s+RqU|I^9;(<>1$sUAEHF#nF`v5oIiM>{ zM%E=lM1rG_Q<=E&sMyoJCtyGN%Gb9x(LK*6n0;d90)`&Qv8Y0PK$HzY9Gh4r2)m~h z{6ZR*0*s5)&JGU26g4Z+5V_zqAF<11c5e$ zwuu&4lBL-iFUz0b%Kx-cA|q`zTiTqVzd@e;)Y5^EQ>fXiWa)@17Dah~{uq(70wXs* zH$HifMO+VEmh6-f8wE5U&@$vv3H-yN*gloIeymwmMyBI@zy>Qt zv+mi|es|F1;1UsOY3X==Tz>BEIioDoajQ`#>)xiQZDx%=-{}_S6R=DZ-{=*DIY~UlBzgW+Bu-4nzs;fG?kY*@m8rFu0((&i`&2+ z;B=x3OM&A+)k#9ykkRs72prr@GugcevSpGIpFcm;mPvdAssp7Fda~em{%@00)-@A= zL?reG0PkdYR}u;|B7{KaQ_!LBO!X5nZTvMmMWYn-SPD){6tip=0MDS#qQ;7|W|m(Q z$1>rp?-KS|C0}`0EJdL{@#s^z)irfY=}?Lo0F-J&h(Y={DCJV1HP#A#E**dR5fSs= zW0f={%3SENN@x3+NGqYmVCAKi?r&C7IM(T!eRD8k@Z#=I-)<1x6{cHwPh?T)5?aQC zrSoGB&cc-Tn4rP&{+aG4i<^|=&(qPZlc7+|GAB^Td^KkuWMc6xp`v_}M(0r~h0SL# z7>koC#01Yt{U5)4yD!GSlA}~{|nvA@nOb#1vSe)YRbqNeXFID z2jwr{_Dx8WcaRF8c-QQIxf6pYBM}AWOD7|EBsiy?C|^&x3sJ2Oh$c7k^2Yzem$w}) zyD0a)EW(?Ie?{!Xs$1r@^F;f*%St>|>vj=MHY;CQ!$*l`+si>IKaM=~$Z3CP_A&mk zBwod+nA6Pf`!>{!kG8oVQqK}j07d;=pXGna(O+w>Wb8e#%4e&uo$Q^&2%7UY!{3zY z&nf#Ii0o<;wy}Bq>8!w+s_$DjDe6B=_~F(W+}z{6Hs9&fm5i7l-SIkzkbt6d%KGELpA2`|CKZ@dU>1@x4OT8$+?TDZA0%%v$ z=|`Lb9K1A-ProR4rOkg3F20kM;vIM^niBoCu=c~uhR7ezQW?zGRaQ~*HPI0BxyDa( zL80!$3_B+s73J9p6^aVSrQO3iKH_wje^TlAGzXgeTnIi$j+8Q%yq~kR-uq+3Wp!Da zD}D!Te+=pzD`Veqg6jE}wJiBN??#4-eE+tReb76R%|qz&o(&b>W_fhiQG5HGuKR1P zt&u#VIOoUHbyS<96glqhFyO&%d;2_-`0@25oj3hyH4drTR|A&a+UM=^w_|1r%AsU% zK$zi(M>q4^2rt!s;khlV(Y`(J*mw{t{z<$+ZvAw2&b-UmQ;E}EyEMNpa=zF{{lDp> zrBeyH-&#RAm-hOYp3#?t^}+9HeV~HjLhp=-McUij8!&=^40$*O3##&wzOQ!fwUFX- zX@8ElZ{Ko2MLRa;HR6emvE#t!JrK$q%`iDmYj6Byqf@zG${t5=6n7%C0Xs!a53-)fr;fKIoG*Vi~Ge<0gwSc_yka8@}y zOi<$&I*&reDI3;W^H@*;_?GKE?~3EMKSNz9BSXKi-i@NtPL4<3|I{pW`hvQ>}^s~T69LL;eWfy`UQYh~YZb$qn;&3`Uw95=AU=pY?fD0ARZS-HQ-#OuC6|b zw4DUvl?Ws8ygX^)ZDy*ri!WY-vZH5aeulEQg{KQM2V=#*m54x)2py5`52l9Uk=JkA zPc&1vZ{N=C^))t!VBy#A-|7Hk0+5#}jQ2lmE4>JwDgqqfGbB|VE&e=@#R|zK0=kN( z$I%+F)a__DRHz7G)yCtaMvg@mM|L#`Zu~gv#K_ESofn0Ig0Kucso`yRwx&Yre^s^E!~Y)c}d^{NLrD!B^V6p2?%2IURo+e990FM zCVKhXja=J~l;!!dWipz@toIw-5ct0clDUbto2Zwd3)`jlyGCMhLLKGAv*VUT<3mUm zz{t_A7@0fFhSNFChU#EGp}8S*f}{xxk(QHrHEF<%_7hqk)ymQA1U53;21>sfNO2NW zpvBjhf05~|lU9%&gv^1-q{plf8Bqb=q?s)-EOeB|BNJVMMF*2z@lz0f;1x_rx!K*rxYWVw961uH}5 zrHEP@d%XJb509z^x!l?WUJ(FbIYN=8A++s&=3x zBBPs;u6;YJBD;zW6Hzqew$-4;ravX}sI7Df4FEVry37! zBt#%4DQjnc_Y<*QRVg#+Es!RV$HLDF!kUlef2R?j`eG6o2fk!dM(YS<#irmasP257TTD1m;f`DmX_WD zt1e9T#9A>T+VG=fHfi@-UX%q5d=KYi@-^N8{}6!i|1KPk$;e~!QG87W$f3J%VMes{ zg*#pQcg3Rc!jbQ=6?Sjmy=!fq9P4PCK{MuHasimHSkz?XE`C;WXY2r#CW~tB^2C6u z3D9Y-Gj1_!@<15|0xUN&3cQD|CzzIn#r7D4?eF#l96PTc6&h-O7HF`!g8TJcJ~e)7tqCMX_}i2vI9! zsdmTe_Rgr5dCMKX-IxqvV`KZml4a8-WP)Il8--r8^|Y@2KGzaShqh*Gj5-L96%O-$ z$86mpu-l&Xf_5%OYd0}o_!urc9z3+5YdHV7&5Bb?NSXycn52oQ%|e_d!R#5nB^7mR2qHh&z>32Od%+A|=^{%TiG zPKN(6&qCe196fIk)*Smj?%XBet|RBq{GhC?j1b@OHLHQoPak7)!=G8?yuuXPV~>xR z`Vo~4mZ`76hSa$Rg&O=9KYj*@6^BuyAv9OU6(@&DqY|opG)N{PKeI$yLgjOxEyui8 z3iA8e4yT9MxZx)*owb+?AXAADpz2p+v@8i&B`|$+EJQVW_9jJOG$3kHgHpN?B$x(* z@e}c@S_t2yBAg!Uun|NnHW{7D{-OlOMg5nUr{IEYh*0yG3%8l)l7}3ii)frQlhw)t z-PAhgA48HrW;sYrjm$^HYDIfm7v9?#Jq!#ay&f`OMRF;$Po%M}VA(gRH@f z^%)WE$al1N3gbBluM4tck9=$X<53(G0$o!R1;s-%Gc!Gfe4r1Y zZz~<`N15AEXEp3Bap%`iP4#;#qwCkNV@lk_?Jne}FhCQlJn;UcU? z$YJaD?HA{Rg%BLkIadQqZ0lz9d+uPRjZyq?{hBDZAx_08iNypf(TZiBt(A3`G8%A7AJ?YGs=d-%+$|bak4APILAq)q*Zn^Q&`I?8jBK*xznA z?;Uuqd$qKx@psugQMRE@N9H*H zi4$YH|N61Cbm_|j|Euj*Q&s)^Z|>$tczj~0nID@S6Pb6TLM&&7xbCS|>S2pIHVhOD zKd~6eAFQjb{R{lTImEIx8kX((n0-2_sY#E}wRX=cdwY8_*6EV)B1fW_kJ@L&)YR0P zYV|_P$sa!koay@jb9M;ZGJCIe*%*3+@53$y`+n|(2+0rkmi>MSrgJMl&t~nzD>hj4;zO zFs!Bgn4Zpi&%RIwX8pLiIR~)qXNcONz*6w+gXx9xo6!*W3s!JJ8%Rb$;ZE)zoosVj zQ8BTi#g9;VtbBCZ2LQwvotgb+D>i$Ufrz=0lOxEyYZom{-{VcB>s zbS*J4v5%pLS0aAB0W#ULW5+GsEwD?enZ zpCQ2q+xI5A`IrgWeea?oF*5VKZqNNkk9@%P?hsHOR3F9z-vaP3!7fw0gM+s}?2oV5 zrJ(%Hef9k&%jCtwtC&`ALu;1?nCp~+!u30MHh|ysMsROvZ0rWnsFK=<|8^WczY0`# z$LMu~IrQy5FST@bvgG9CxY=l{t7A9g7fVxHp2t;~HZ?XHqrZr6`#3byLGQtDV`EcW z0V^Tm&$f1#@r7BQXQ-qfT; zRvgxol$`v^X9L6YD?iU?xvQw8HvtRm+rNJehI((^y(E_$?`ub~Gj^;Bgy zC+B*`duvcsEqpgA}53Q|jR`I6PR# zFGVhVs9A1d26r`}2>d^wY7rW!`$0iBFyOQv*?{6r#43)4ZfgD90V$1Y^CraeF~wPq z`b$Mm&kqK24(>G2oWTkagNmVH`eI62nooGRex6O*vuA~yzCj^u8z9-P(_V6wK}ku; zZS-Mm?7pF)q3o`%0*8vwx{Z57T3TEE2I~{|2?=e-mtUa8nvv1$+11bx=b?@M$A+VB z6CB=aS02HWmMu;(VdZRs+vOVFx?isj3JEbmC4*2TBPF%M-ofE8v3G8cR9?u+hSnxzq7m8`W54v+b+Zh;Qb8_gB z>ef)MT)m2cVg_K5Y@PiSbSYQj&Nw(ZU*SG$F$VD@&nYx;AKE5~Z8t;i`1x&g6CUmo zV)($x!y|L%%tp*DZz^0&P5mriBIf1yIrO33BhU zvTGnx_D9Jv^u*#!8gR!;BdxJU!eVc1ajQ{JU@ij^<~VpzAgNn` zeHx`E^nAXFi3hybvU2fpbJx#r+Gk=lIXNkF;sh1GPvj?x5b{MMYYaq!s?c;T2k(6e z9&ei`p=)pGwzZ$zdrXffN7DCP?73^{ZL~MGT(Y;9!VKqA6%OPGe?SC_@i5Y~ceW5< zVq$7&Za%q=3et@BLMMf`oy&_yMj%?s;qJd+YPtu!amFGw4NX(0vIjIpS-fixgy`z% z-176IZZGpJKIe#xK%<(2m`z(d;Bz+M=7cUzr7kY)@6)&QeL9f@^xfn zBV;oV9gxl+Jb7|&YRXwdQ_~_uo=MzIU}$()Raf_RW##FS_JVB)h)1D|{SaoKgBfiW z+_#LYtB9zm=yTf+Q4|a@X!!2$A4;L8pl-bh(x4j-cl7GiL8Qb3MX&w#gcbV7^ZJ$^ z8VH|Hx*V&OTKnb8El@8ju3WOFr>8fD-r(Yg22CqlTV8yDgWtP%ve4|DN9$0w%QC)Z zd_omdraq7rwHLW;h6Y+nNy-02YTiOy-(C^SR7hJ|av`Q~{1n6#Wsd~-08ZxXyM|;= z1jIFItJs7koLqswRn^sFpCm%<7l0IX9&>4Kri}W-DC5!SUgPNO90Pa8AkrGa`F9PX zmJJ+98eA7HTsXe&FCaxaxN%2ki^ti7!xdY9eBHF>z<~oLSxb>t1Nl28+>a_n9!oCK zv&XpNmzEaNV;w(ZV(Ec#B-GdeaFS<6Wh~??bj+N4B~qGNTlEg=X0F1Qf&})53tDY2 zog3nAM<#@6eSu7CYKq;1-|0^dKxS1K4LGq>8=$mYfy^g|FMm;-81FJlYP6(fV&Z0G zV)EG{g|9`~fcd0u^mrhcdJCh$n;IG#w@^5*LrJpd#+DBsKD@@Dy|k>XAL^+Mn2yDp zNGW>(J(0LiV$9ty#L!J%LrZHY$B6Ns?9BZ99ZcljMjH16(jB$jhVML(O4jhhcZ#fo zu|Cz{D)MOUq(f1?9Sx4xn01UxNU+rYwiETUQ0g!@nkNUtL_|bh*Vn6)vqMVHw61QL?vY+g8(Z7$7y*`wS6h##f-u7@_{<(hRt2ebwVyxlrrZK6 z62_yvZuauSkdOni{>+?Ul-{CCz0r_e)<0dZt+iDe{Row0jn{DDo-+m_hzMIw$EzW( zsdLa#<2lO_39z8qxw(Fji+7KV9E3`ZjRc}ffs4F>ZLI26ib_h`@fjU`{SDmQ+~hMT zP^2v^k}02U{MEP~eZzNFxS0(xMm07zw)S5=8k?D^CLkzy8#Vj}IAM2B4;vYRGSOqT z|E{i`=4JLVMebcgO0D@@w7tkO?EouCY~uucu(BlmQBcr2FE6j|uU~h=^rbIfesQAU z-Cd?VH}G{Y^f0tFrcQ^a1Q(#hrGq@>HNI_z)b$IG&CaBuT%rYf++K6+_uX|+r(Sh> zSJ5~}I^tO@OGA1dygWRLSFT(kcJl%v!|PhPfnvcupR~`2d({cr_E_fh%swpr!m*%$ zL2t=lJ@q>f9rp7mnB9$u*?Whlg;OqGZRdoP6;0nl-jTQUYQqmjgO{ql`X{bFYp%mx1@eW=C40Lv3=p# zLJoTXF!blJz;m(3K`cm1Z?&fTbHu4@_g0W2{@a;^sQ(3J;q=LW{Q(U|tKh8o+ zyYn`=<**CjCr&9UmS0=F0U`M=BIkEZD9c$}A0!$#%1e-M&&_IBMuj$e8J#_Q7i8nh zwl*d(pD*E>6bgJ;C;!r(NUtT=gGQyyxw*Od9`m_VB2cdQ;-X8}A8n--H~4c;wOm|W z(Y5AV5I0;8^77Jjc`cWWOi8W7Z5l&mo^Rhv4YA>tUAssw`10k;Zb?a2D6A_WJfa2m zhU(HXkEA&?h^q~^(b9^)BrAJY3q}vzeN;?LfyKeaMb5@1ZDa~%g^z6PD=>T%3TSHI zn>W{jIHaQN5VWMjIQ&LRsF(|{^LXbbO8%9fdt2Mu2>0~K{?@I5Jx$7PYw@}qg3pX~ zl?9}vaLH$>sH#$-c5fLS9vb@G)RbVeh7w&+uoLb?@kU$Ka#@0l$c5K)aC4tjQlf#R zY7<%(?<{d5T%>|$a}kf%~8@KwZpOI9m$a|#7>sGFUgojLgU znmsj<3GGbX@@b#v;*1H)|Naf#xQ}fKTJ&S7W?o z`M8s`v@&0(8w_{9D@aXa8=HIJ3K>w>Uw`z79mIy*>((FV3{wThPDxEAf%QRu&Bu>7P{xF>+8g6V$smTV zi>AB@2-oMCnXf@XR8JLSWs@>8x4p9VY z5F`YNa}T%fy}#=^*ZFbwxBan}&UoMPj`74D&%C9gbbRHCO)DrA%1XHtN7X14dL0Ud z&U@JsykhxPfQJ7qayTrfu?&AMFFSh^|K8{*tK+C{bKcR##Qq%Rg0+p+IYEar_UF!7 zJDA!yjxZET<3obvLr3h-nK+u+SZ~!ZvpPpnce38PM{KL2iT&0+yY}qex_g)Su6^Pn zyS5%u+^Tp~L-eA}MG9prMegV!jf+o*+Alc<)X$ZSR-WA^ak7cJm21$1(PFEm8RgK? zOrOMX`sejpSFUDuYwP*Qsxa2)US(drT9;==){#15~)MX%R0aM2xcUiNNw{?plh z5tF^F)JJ(nJ+?|cmxd*$o;-^Ceqicj&)s744Bf?8?iqa_@~D&xM~dhvlwG&I@S^bs zIz|emXczr@3dPH0kpSL)y$riUxv?e{|NV>8n2ti(aP0r@3-6P%84rF=HlRhD)keo% z`u=s3XN#(TY#v4QS4IrGPGfHdHG?OYjbx#AN2s$4fnIbKhPw&6FF>Zxo8+=U{_+nW6wS=c#2vogZb&vr5Y)>w?X3#CUJ&F$eextBrC>&0 zFUwj}MdjYxJDaslB@*i6*(97z7v|^W<>jd~8tw4m!*v-J+L|eb`Bgn{N=huIhTF-- z)YR3TzJFD`qE=4dQSaX$X>qQP*KEnK2#VKENft85ZK7ZB=J1H!edaCC{{2l24dxl< zIINZeciNDHHqF`Ul)!p}d{@n#%E5J`UrOp@RXJB~+;enzczE&5mlumD+3Psz2APJvbz3a9o@duC6=j*Z^mvW9(^Gj} zU1!~A`eMjhBqb%se|+&{jjIr|YGK|mB!BW`ZO6jA3!CIc6)D&8RClw^ie#h0WQ21Z zj$@OUbm&Nz5AIL+~)6i?{DCb5<9Pi;HV{e$F>9 zKVQ_mF7|K?6TfDBq(A5JejCr9Pcp6B<8R(vrWvd9^5{D;4-b!c)rcE+*(9QG-~Njx zT-?^CPj+ng?%ncAN)Mhqc|y;?P@ildffc@Qu*N3prk%ci%#S@OK|95~Ff3hVinnqd`>2vqQY>4xB$lwdDQG5WVTWZNPZhHo^|s|Wl$DpO z-`ONumu1sIZ^aQ^pP*fCp3(A3)6g)Ota0w=)BuhoUO(r#pk-x~_GpeXNtJ*A zE__8ZS-;u24RQSW)%BKClWXjky4Fk#G%6#vv~h(80;j@JDVkcoF7-;%%fi=>Nx4nd zrFhO+Ac-x=R%e(88iYLAXVXvG>aLrhon% z7%kz@zQ03pd~T}40(X3_oAlPm&&~hdy-;jrf^vxT*tc&gxGxHB45Bgq)Qj_*#2sTF ziF3yIxmV;k^l-^J4K(BrU~^JZgbQcYE>s2!sy#y(XHH0>Vv)a8M|;f8 z#3?-5lbPA!KK1M_hfFduA?{4c-OMjmt(o<1Bl$6;&|0;m`(wpcRyrsIhL^fo^mvx7 zSW)BB;U2?z{H|${7u|Cm0eMBmdTk^3s864a?`+)rR;;rI-Fm0wkA!Ypi-pZyK z2}Ibj+Z)2Fs#N5%3*m$!>&tgx8QFJGL7+Yac3-m@0G zJl}T{?L!r%*J-Rv&Y;l4AUHU9W|%e-5EK-H=1XggK&L?`4nXx7npyGJU+LEqLNcYW z8)!x&2aJo0i~9n#4Rc~LT^w))QL~z(>wRVS*?g=+uR#h4(i#uX*xa1%dv?vx&=3Vs zOzn69&b4FGA)rufaAsnFI{$NHuKJHc)~WAkVFA)_GtFX=%o!Q7JX`bW~JhDgqJYJn&%8o;{AAUoCbYEoPmWY_$=!Y}(bDt~$?CKHgVr zM#DEgAIBS(~!?sD-WoDqE zhE=QB(8qkelMe){NxJJv^l6>Q(d#W^+9Nyn3fZT1MWV1J(<%XQ!Q$oB#k%>f88=q3 z@%1&_^e#SKeRB<`wOjteN|~wWhCDLN?E}$IrfPr?jDxPsb7AeebykfFM$=&+RXD>aIa8Y{n=+W5Zq!zy5_~V<& zGs#P8X!f~ZM3q9M1?8OGHa@CLH8}zd(<)dni4p_|b@kdc#x^<~=INJjxf9)*xU0fl1{5b1|TYP3?*ECR!BNvo>=SR+mqNmp)bh>(a zEzI=(k$3LE5#?&+IvQ7-?g9P-sbt_G=@v%nJh2<2L8Ri4V z(=#*uC@P47C#yK5^-Z6ADe=SeD+TgtK381&fL~h)C`Lm|>%shVrDuPCeZu14pa&14 z7c;K!MPr&N7WNn|YOon4{96`zO= zW!9>>)Px>l$jQz&tqhiOu2fb~s6$_L>?-x&J}=7jz|YSQiDB8wUI)zm@#DvqY`X~H zw?KdYFw|=c9Nr2hChcR#Zm6lLIgaHog%PY52 z>1h>;XI-XM>tUChL6N`TR&$t@F5h+Nw8XPpUwh>hj3`j~eLz%J~um{t7Xpe1Q` z*4Ebj)e$A`M&C_FM#7CFGCe@Xj#p)`aNw93* zY|-jfS6QhH9N6EIk)WGys@glUT7GKkS07t%v0RVmxkaN8fc#iyp~X$A2@Y-M>k}Ih z<07xt`J-wDwK2*A;8WPcxg+tbC|;f2;^)c&c#93*D3iV~;XGJX?8{moF6$d2y3%`T z3f6(PQ-hQy9XuyDw+zvNb_=eF{0&FQWY~AhWB(LUL^7SMD3txQwfnAex6kwi0#Olw zh2#tZKHt85`>;jYZuYw`E>xNWgb;H0;mN+C3UXiMvGzbi(vXMuS_lN%+Zi6zo7dxs zNuR<-y#u)cBv~tQCde3#UIW4U{P}Yd6SNGd1p#8*dx%i5|M~WjYeZ^r`(IEJ9QcRn zDHryc*gYta2ys!iYHf6=uZxl7u$7Hco(!@Gm~5zXGapbBRbe^r!4u>ax>TQqa^dZ= z_20bOEwZVR&tmLH;@W{F=CqcWS|Q4a+5;(?-EfieOgJvwDd%UfU3{_z56zn}EX z34ecoRAklKXhmcX7x&Jct@Swoi)TdXviB_#D5ATw<$xM0K>f>WOP$b9Aw_5rYUw94 z|3hWqM`Pi96cG<_UM(*!_yOEa0Qj?-W`XLu&rkR0(wvik#9|^Nx!lLAk1uXP!Y3FN zdH||czvd|{#p`&713`|Kox{|Qx|cZ2~DH)bXu%-is8&yxv*xY5#TL~^%H5}rIJx)y`zlfr z@+5y&X9I}#ec0EbaeW0#*T6sml7U71>i1GkKT|M3(vN}j3DwWe&ZZwioB3K9k^)%s z*focJld`fhX&X=>0_)8Kxdcx=KguTVr~>YXHYS6k=i%m-C7lQ?VJOFQy4+~IM`odU z*ww`?WZl}e>BcYV=b-awfW~(B^u%B(W|M=>1Ry*RHkwyi@SsdC)A71y=i<^B;U^t` zA;euP#C`ISkWqnv91(`}4Gg-!evQ7rQ#l@3MYrH`j>ilG!jv9%2VhVw$CD$nd z(YP7wI)QGIVK9Y0E)MJtQ`OAP|@~9Tt$@} zphImMuaFpty*xeE-;@@im11b+HXLN+5r>l3YA|9w^1Zczy_$(gP}Q1o&ZDvnC81ZvNr1Xy$iN8%>l~ZaKuakFPI5KO{twj0z3e+1W|M zPS8qnL>X)S&h9yDM_cXGRl0_BK;2a1E0-@{w$@YjW0xY36Y)XhBbP*XkRY<<(3-{^ z7G~Fe0r5};143$PW?36>@7U3QjY;|*7pY+-ejM76rnu#>ovHZ^1fP=HC|8qqrX=A0 z{Rn^u=Ir@rAA14bwvhEpbzm&Na!KMYQer( zn3*Hd6G|&9D;pZnU%_gS6#fYb3DHW=;=w=P;YA`re*65|%4%+IZfIt@eZ*s@k&zKJ zmd|%55&?+by?ZxP8!{g$GuNj)Ghno!8%hcfl)-p(c=xe)JAr~`7iROY9e^z%k$T(q z?%AV8M@L7sp35h=Ld?322Z9~qBo54(8fo30OM;0gaYPgX@k1j1;{l6`4z|WNEXXS; zXsE1qIYeK4YmGqB%UPmhg56sBNl8gfKt_p2gAljxy2;fCX*E5G0aOi@?*YLxg>yqT zV9B~K&fg(4l!z$?KRpY(Iy@KLMXg#iP`dcj9{@e}p>$FswzR(?=?S?PUR(c= zJz#iVdLM2?+BME^F!(}-ZidI4D-{H?}xt9IawNGcG;6{^*#BrSpyfSc6N z-?-%(Q;f0{xsy)4IEDueEG#TUZYJhB4V2-yz`1K6d0Acw-L-AoHrJty#&jShd_AFP z8X8P5!~sokZ0ok{FobdZp=(RfQY2hj=sN%wG8}r8=Du(&+(TP!7xieV@OSF1I<#_= zNEI}QoqY{FHAF)Jga$^iAZl5=MLSx9pib(+Q>RX?U}WUC&8Ov8;&2HL6m=fdAr;`# zrAxWaLyvbTKIQ^J-+YQeMUoU4Xj_?IE1o5w`sz|_T8Z2G4 zMFmA_!q0Q|2tmDEX5+opy4kj2q%tze%&MqID^%|cxg3tF)0SBX9Ax_T_QuD%Os;~! z4psctYzclMTK35xq2}QmdU+Sq2xgbd%5>Nd=Kl*x@9R%1Vdt;1274myZOLC7Le*oG z;?PfA(PusscrjI>zKdR{*n;XB>oW2&*S_0lJ13{;`S%+@%H&9jSfu3dkn{2JQ9FPB zd{~5bO@4kQS)RJGRMh9!-U$GPfJXWH`uapYp>370Zrcy_qAN zD`Kebo@n)3&2qlBt5tYMA?xwGTdD!*8QJXklGHpTjveS=%kG)%zTm+#6)nhNESq)o z*fI0rwj7{aL;sYC7@0XaaQm&2bli(Lb&43Y63X%yItD#jIzP7C=sf@Nqdi}B4e-7K z)a+xg0v#>N?7GVWXFhv#+}3A0SiOJVO7h_HsnHWXdR0TFmxDJ8hV&d`b=fGUEIl1r zQ&Y2|ePX`HygK|k+=nP6D!{J%NZ~?3ngv6)r@|W8FB{%oUi80Yxj6KC1qK8}5)ccl z;#{Jxu1+vdf?AY3LH*InA=VPCYu0Q7kAgZ^25SI;$3(NFP(F2S5i(j*1o$4Pjbp(4LNRkhe3Ht{i~UslSeQTt7tGT?1B) zTvjyW6x1FBfLyNVRUonbix(}5g+hsov>!>>2$Z=BcN>;Pg<7(56A`oHOi?op5>h76A(&mC6%`e2&(Ee%LLKePcwg=Nc<~4}HiHog)TjC+y?s_~ z*{alZ)G;jL`L1jfSCL_owDs0ur>V%4e! z5QFZ{v5wgv2}X1Bo(n%a2)WCj9V?eVaYE(9iJJu20kgAyF+zYnz4Lkr>^v`O8E?!o zU~`{!|3#r(jh z8EO!x96>hUD`irSc(3B!zh8$KZ?G4P^gZdf)ZM+G^EhiLw}M;Gm&Uwdz5cs!|7VhQ z$;lRWLG(pe4h$~SLQe?~53fr%tEKNb|NdbZROM$gE382Dny9%x`v-e7!touM#PoHi<^J=~eg^z#l zUUjH}&Ly6%|M-Yum*EB%7nd9wjh~bA_4;V^obG`Eehv7u$Z@gXm}QJ0~2!zTbH6{Q0uh z)>db0u!`%3J9q6$Js~Xgv?*w#iw%Q!13Gc15S3~g z`A5W72%7acv=2`Ys1vlo_2HR>wvp#t1|N(!R53F!DSM}Lg`45I6Gz@&rd|G}4~4?o zMrHx_*oNfhRWOIWXccLW$v#Q>RJ6r;5sdlJ$nF1$0slXeX5W2Q;%I=1C4dHGGMpck zWo=3+r%t}x)B^xv;was`dGl~9gr(y)d80SLoQa>unsJhRGO3hqAg3o-z0hNBxM|1s z?T14==W~dQ&YU6fk1MJFJ^i^K0(K)|J96Sq{QbN#iFweloi(r)+tcwg#4Z`1rUZY59_(*)E=s3s?=U?ld9i@)K`inmS7VT8q(gMR_YQmw5jSF^JVX=`h%8&J6E)&H*W z<2_7~Rqa}nh_3ZXpS&fcaRUgaf%j#&O<9wZA&LxQCIkK?$V&au zXSma<2<$j?;2u}Un$zQ zp*@<8N3bdS<#d`NgID-2v8y1V;`x&;7NifsUTn|#@#CU;JFXXS@hyD=1|=q1EK=ft zgDeFwg(EL|hK>v~xZ+F|3`mtQ+1MH#hOB8xH@hEKPJ|?ZGn_gW4T;rY7a-q+Nzoh< zU*h&rx2ml+wYA;;{k@Y%K_v*$Oa~+f8;pkpNEj_d#y<}&V4P%_uZwD}32U@#WF&=X zI8CW0x|x=b;7^~9M@o9Ft<)(ZLLtb95U@v+vn5pRO%l$D5ZDx*NOE*;r1^eBGj{$g zPkr_JHRK?(-@7VcOoUN5IjZx*BRWZ1g1YG^@XGFbC}Cblc8SUUc)66rv?}zF>*wpN zWC75E`OrBQ=BEmON9^sziK(f(h%^o}ip(znxW&U^#l!Y?JZtLrSIePX#j8a`$dS-lgwETL4CU z*#9;7skD2`u-)0Sk%&}bIgoQzj2B@!<0=*;X1voxbqc3H*rYoRXyuQU@n^~okxQbB z+r;~Zbk{1&Xkz#XOa+`V*+4x9v?T+$>{HwAcG(S>T>wLe^2*1>brdd9AQym~c3N86 zj@)b;te@0G2q;O!CZ@tRa2@Xr15@NfhfOJ*&&5Ag1twnF*|COw|00AkU}CwbyuAD* zp2N8WhN`dA&MCC$STM{?+dx>b_={Oxtg4TnUt<8Tx`^$kgR}`CP+4k1BduAy2t?I6xL#uQAEbQ!{Df1bj>1hMQ#a6fm0-|AT&>0>HKODLy$v}K zjD~Ncms!9Z0JJUx^8l$MI|4y83{;wkxjS+x@_$7yd&Z&uH^t{w(=sylQuaVF0ng5h z2Szat;_B||dic8nGU|Xr!RUEhFw@V!l1*HLnvN2P5sW9lKhV0A3}6t29(qIc@^w2- zP>FX1HeU6QLK{|Ukt+RjaIe&K*ZsGx^xPXQQlD~Iq*9n={g0=TZaP2rv%^lkptsi) zB5H^AePx8e1AP)J zYn-7=a3wLMAVN{Cha~EAoh+_jzs@oc9pPoButr=adKn$_^)%Wt!S%iv<6&OE{(L_b z_gnR@$>4ANKl}Q>&kCNQ8^8ixd!D(B`x?z2M#~Ivv|#i`#{Ov#jQ=CsAVUQ>@;gc` z%zpn<8iu#)7kcDFLhkDC=R=>As{s<5Xwb9ybA`<>34+D6Uwaz-yvah6#dp)ZCNP*0 zi^k+SH_;dv9Gslh;h_gdiVR`J31OjIuBJU}<2D0uA=Mq-jaW&LG70OC7Get6FxWLb z+w_hNak*0GpxL1!dk;7jNJ$@C9M_7KD>ae(P?{~Km;k;+c003v22vMs8P>l1Tl^UwB2pn3#v&OdJiE zT4PjLvlf$;2hC0k7#rkWR3!z)m$8roAC=#C#*9ZqB)v#!7*~z*2Pgq>CUcHBI^k5& z*En^B-@S-Y2OvZn7T=$CNk{bbXd$$mPgOo!QW4V;SOU-X?VXucaN!N=tjYEq^Ji#a z6wV8U0|g0kCg_$cdDYP0bFdkj7;2)3Cex?YKhiGrl=w+8@T?F^!p84PELHy-GdNJ@ zv(G%iEZmgr5MryyWW#bD$S%880Xw|ZdF+b_t%fG7ScfltV%8x}kg*Q#i2PElBgjky zE8C$QY#ENGPl-=U0ih}=s?H??b#Z5&|7y)2bc%>z8Xu<~QwD^U zoSf|31N-?;TV~WDt@rZfJTl7nCu}z*0?4mT#%<2pE=s^AKi+fB2W+4J50dy4n>^AK zetVjOcdG4Cr_kaXIZGM?M!pwnJ#zLhHdyiW9+{d|)F%qLy#v2wO4<^}isXJS$%UhN%9C?ITW95dd@D0Zs%FcMf}{{#dT4X&^KH44`k)}F2{GC z22{JJ5}CGRelm;9ToH#AKKbLkS*f3MBc8%IX(VsUy!%wU3pElA2k2F9etzEkV_psh z5>5$-VpDq%X#{7v_C*DCb$3T04PksAA>I%fQB*f*djV}be{QlBYDMN8Bk}w1Y}kDm zAf31zP=b=_7?$wt+}RV>xC|0O5U^Ci+@xNhpk9NsC1|71?b~&KFJGtNapvSHaD z{P1Dy!-pH(k#ZQs&;+7jS-m=xbT~e~{;-B$*FmTPaW1J`uo#_o0pb1qur`e12o<{eR^5$nL>9T zi9)6mK^*(Cpv7CYzJDf>72Dnxgw`347D=!w3#Of+Hy*{TQyO8J1Q*~DXI^v6hZ1!N zGPfq&E_QQwBx?@K0psg8ZV>4^ER6d?*;$;;roHEHfAsii1(V_surUFP-0dsezz1vL z4QgWIR?w(GANf|SzZ<8tLw1!8K1Ss-$g!7&k1cHL0Z=PD)LB3t5>6G=yb~u*q*-|p zkng>HX(DkcLEj$BczRsPkMWotJ&ur1j{Svb1MohEVgq^Ak6}OsXeUde2lH!yLd^f* zF=}02nlVajbCX$xnC1RZm*a3@ohac3B4=7f1i;gv@$=h~;Y!$Y(YbhsXA}?^sF_xR zaY5pOvbzoL{)1TQ1~7~4e&q}%1ml-5v0MJ3L?ywDUyw?`oHC_Cs2B2EKs^v{8E#Jj z#-Mt%PrA(G^{4d=de?}0Vl>;yI@2xe`3L?xu7v$SBd!%NA{oR1XU>?5g=lcl%mk1P z-dr5XI?SC?10Fq!!(;<-FaNYc`UZk*fozmup9OMBjg@kefwl<@i5yptC_*3%2Sq(29qKZJ!hrZGH&ueVjefn>L zfr0M`38oTrF;x+?iN^^?0&50_UMm}BES^ItHTfrRg1Bm4@$>+KGJ}mc4gh3--n8`p zd7Q&N5)Sw;quB1t4Pr~dPO*4L<^HoYdNK?Nd`L(J3Lh>P4_@iaN#JtSl{DU008s-N zoFyYIM2p7Z41i+z+`PH#H#u9MN<0rpz{Jx8BzK+e4yun)=0NsV{PXEZB_Xm2f$88x zxvmrOciE-lG5Gqg6C&S*iS0Bwc#0S*M9)Jj1fIX}&!-hNtvU$N7&>}2Km{SJP+#JT zK9ZkXR8#19KFd?B%b*jWFY>guEei7DAwElO*%>}by=eeYHFC*G`U|`xN%M)NS3QUY!0tP6E{xupW8epLBnltKgoPrIBLDDO zd~%%w&DR3IXQlb_%dyCEKv@X=p?urd3S4bti7$YOT7NU)XgTtsT3R%0eGeG_Cnk0W>s1wKK*O8K;yZPDaOfkZ; zfafx8I+9T#n6uyh>OuFuRhi%5T>giwouryGJFs2+Hr8SBDdy_ctAMvZUWAMU&=I$q zjE;NJF~FPTbiVkb{J|1?=Y3@vK@9A(-7^GZDbT@Cp5loS|}6pn`5>Rb%#gk ze_?RwKl;tm|By`oE5C~RkM5AF#+phdZ!r4@wwXYF+JL@j7X&ZysX`p14(GAkD)`v| z@dF`!T42^4b{_BWa{%piyqa|U|2HGROBFtmu}0hq9f%sB&i>%OLpr|!+UZ0z^ZifM zSW+=j(%s#C&gFe;s~!~oVvRMx1jnphY&s<|yAj~;Z{i{#C1p@hP;fmnWbcK$=yBh7 z$up?DN}oy0<}W%@8?R}n?X}hv<}`ej@k);Z0Cmu?klM?>l{?Oi6M+oZ0b!UI!|)j9US5MXf)FOMwgQTPAP^5RvV}Cw zHBoDOm|3gY!|ma^Ta;VKDw$eY(QXUnwmsI})}L+sESEGb-S4~ST$uXp{?_*UjrY6X z1_fyHJ0M@HMeE*q1Am(&cxZVE-MvAo+7B z5$snb2u|9bPepxBVdq1n8RDQ{>=3Re<_<<%g%!oOhA6jfTvbO zy>ZBMI{10J6Xw}m;IS?P^mp+P$61Bo-=lL-KacUWqa7FEXtSYS4dQcwe<4j1AN7wnu*LqF01N3k8w#i$qZAP|rIS};By%;VX3juL_A z4M~9WGdu@l7LY&b#dyVHpNw(f@tn9(uM_jvh5YlV*8=-H&@?gI<6VJO1`Hg{i?{(n$oQ?Swhd84#ACl01BKCIBx5WeaUc>YIffwUih+I;S2gZXK&psdEw#K9Vc+gur7tc`&)Qt~($S3;OCIJ4^ zcCa&+gy$d*@_vB`;~Pcbzbp;>dKU9+P8#r7kcV+aJRRA{uLJ8=8si?&53Y+=464tC(#a{g*?wj+;aT0F#x-eJhr(oe-g2NyCGf* z59Fa|HrkIzd?^@@1ejkbKz~*QeRUj;&e0g2IEZAb9b*FjKGcvg>Z*sOz5OZ?#@(cWhbED?n5SU+7yr87? z#=Z1}UZTk7S~^Vr_UpJ;gUseQkC z`k7~++yCnWzlr_r^Dp$h_|n1Oz5EK>e`w(MhYuYYeD&yS$Bv(P{f#$IzV-GyL+}2f zxpz-u$=P&zn6SeKf`G8GZ%)RMJ<-cX@a!O zFql>v{0v|9N!OT>P#j-IJyDoG-{4q-X(!7~!LonJ2iKzrOye+2(@3xYzRHw^V_F}?9~+@-+JkgQ*Y{%a8bm`}PcteQ@2IjCMoNpHM8LO4M@>_c7={gXQ+B z53XCG=r@b||K*de1AC?O!!S%F+LQH%!!h)K=aa5?UIy2r(9apDCk%O-hVcl;GI$B) zB9Z5*D5pZbQOM7?$jcm%j#sFM7?S58Uf zNi&mj3UBA=F~M>=$=B0UDOmKAH{Dr%q|Ot)eUsqWQi1X-jXT@8qU;VrB<=3{3oKO8(L}g2?$+-4tL2l)nQN05< z?d_{?+&X=EX~Ox`#(^h_4>zVhvil#$nBI928|&{r<%qgfz5b3_`>eX*VeaA7Ms3CV zs#iW)9kH(W$dG4a|5BwVY2e~9_4;i~+~IHcZa5rsu5YNmuX$CuUOVt`)#)1!%pBaj z=ZlYaecU^29LPL2^Q+!H_NQhVJ8oNF`^xF_>h%LG@o>CeD%*BfF4THhlD@ zogaUwW1aoole;f}d8Fa7*L$>AS|91|{=)h4>BP$;45!6{u57DW0|4J=b?>wMsWg6P znyLH;nh7Q&XOy8Sh^eAxYHV>$8$lJ|0>P-1WLB>zY7PR_!b};nOb>nGg4M++Su>;j z(#VEfxvXPDuXGkU$HA!ERy8-&v;v-fR@cw+8#DN!bW{0+P68!^%Ls59i6si+#*QYb zt7@lAmDP2$(qTbsJ%LuiRRlv*(N(>YtWAJZ@JtdL%_Oo$g8GqtT;{l0+LNjW(LMv{AgYQM9xXuqRpa3x=zaH|aI97}-fRl53LE zR~REQDt4|yh-8Gwd(CW&z{Y44Dj`%IL+l);rUc1wr5HnA6w+~cfV|x^Pt55N$O{vb zSGSXNEW0~TFsnjTNsP)u3JICwRG7M|w)!q~f0t6XOUa)Q`I7=UT>lB2VXDF~ai=4% z@RuV!*d9ACzBX3$HABWQu-xy?PB(`PzxNOp0GI7Hyf|4PVI0CX<>KuL!>1!2~T}B;&{utrYQ0z(nQg7!Pak6g?Zk zV<|!92^Eo%LPitTU^1-bhhk0Rh4RN=EOjURoL=`{9+%DV2iy~yeoV>LeL{8In@@Fo z1!!ty6uHTc3lByVfgpF6SCJ(UUd7Y=+gyU62qVM zO^9A=InW(00?x(oXL3#d-+9svQxKc|ggM<-ABiZ*O}$nuEiGP@0V$9d+X z^63Mffd$0g8md&39;D9^>pJO}=dCM=z zlV!x*`E*^=0.14,<0.15"] +build-backend = "maturin" + +[project] +name = "raysql" +requires-python = ">=3.7" +classifiers = [ + "Programming Language :: Rust", + "Programming Language :: Python :: Implementation :: CPython", + "Programming Language :: Python :: Implementation :: PyPy", +] + + diff --git a/raysql/__init__.py b/raysql/__init__.py new file mode 100644 index 0000000..b608318 --- /dev/null +++ b/raysql/__init__.py @@ -0,0 +1,33 @@ +# 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. + +try: + import importlib.metadata as importlib_metadata +except ImportError: + import importlib_metadata + +from ._raysql_internal import ( + Context, + ExecutionGraph, + QueryStage, + execute_partition, + serialize_execution_plan, + deserialize_execution_plan, +) +from .context import RaySqlContext + +__version__ = importlib_metadata.version(__name__) diff --git a/raysql/context.py b/raysql/context.py new file mode 100644 index 0000000..c44de4c --- /dev/null +++ b/raysql/context.py @@ -0,0 +1,255 @@ +# 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 json +import os +import time +from typing import Iterable + +import pyarrow as pa +import ray + +import raysql +from raysql import Context, ExecutionGraph, QueryStage +from typing import List + +def schedule_execution( + graph: ExecutionGraph, + stage_id: int, + is_final_stage: bool, +) -> list[ray.ObjectRef]: + stage = graph.get_query_stage(stage_id) + # execute child stages first + # A list of (stage ID, list of futures) for each child stage + # Each list is a 2-D array of (input partitions, output partitions). + child_outputs = [] + for child_id in stage.get_child_stage_ids(): + child_outputs.append((child_id, schedule_execution(graph, child_id, False))) + # child_outputs.append((child_id, schedule_execution(graph, child_id))) + + concurrency = stage.get_input_partition_count() + output_partitions_count = stage.get_output_partition_count() + if is_final_stage: + print("Forcing reduce stage concurrency from {} to 1".format(concurrency)) + concurrency = 1 + + print( + "Scheduling query stage #{} with {} input partitions and {} output partitions".format( + stage.id(), concurrency, output_partitions_count + ) + ) + + def _get_worker_inputs( + part: int, + ) -> tuple[list[tuple[int, int, int]], list[ray.ObjectRef]]: + ids = [] + futures = [] + for child_stage_id, child_futures in child_outputs: + for i, lst in enumerate(child_futures): + if isinstance(lst, list): + for j, f in enumerate(lst): + if concurrency == 1 or j == part: + # If concurrency is 1, pass in all shuffle partitions. Otherwise, + # only pass in the partitions that match the current worker partition. + ids.append((child_stage_id, i, j)) + futures.append(f) + elif concurrency == 1 or part == 0: + ids.append((child_stage_id, i, 0)) + futures.append(lst) + return ids, futures + + # schedule the actual execution workers + plan_bytes = raysql.serialize_execution_plan(stage.get_execution_plan()) + futures = [] + opt = {} + opt["resources"] = {"worker": 1e-3} + opt["num_returns"] = output_partitions_count + for part in range(concurrency): + ids, inputs = _get_worker_inputs(part) + futures.append( + execute_query_partition.options(**opt).remote( + stage_id, plan_bytes, part, ids, *inputs + ) + ) + return futures + + +@ray.remote(num_cpus=0) +def execute_query_stage( + query_stages: list[QueryStage], + stage_id: int, + use_ray_shuffle: bool, +) -> tuple[int, list[ray.ObjectRef]]: + """ + Execute a query stage on the workers. + + Returns the stage ID, and a list of futures for the output partitions of the query stage. + """ + stage = QueryStage(stage_id, query_stages[stage_id]) + + # execute child stages first + child_futures = [] + for child_id in stage.get_child_stage_ids(): + child_futures.append( + execute_query_stage.remote(query_stages, child_id, use_ray_shuffle) + ) + + # if the query stage has a single output partition then we need to execute for the output + # partition, otherwise we need to execute in parallel for each input partition + concurrency = stage.get_input_partition_count() + output_partitions_count = stage.get_output_partition_count() + if output_partitions_count == 1: + # reduce stage + print("Forcing reduce stage concurrency from {} to 1".format(concurrency)) + concurrency = 1 + + print( + "Scheduling query stage #{} with {} input partitions and {} output partitions".format( + stage.id(), concurrency, output_partitions_count + ) + ) + + # A list of (stage ID, list of futures) for each child stage + # Each list is a 2-D array of (input partitions, output partitions). + child_outputs = ray.get(child_futures) + + def _get_worker_inputs( + part: int, + ) -> tuple[list[tuple[int, int, int]], list[ray.ObjectRef]]: + ids = [] + futures = [] + if use_ray_shuffle: + for child_stage_id, child_futures in child_outputs: + for i, lst in enumerate(child_futures): + if isinstance(lst, list): + for j, f in enumerate(lst): + if concurrency == 1 or j == part: + # If concurrency is 1, pass in all shuffle partitions. Otherwise, + # only pass in the partitions that match the current worker partition. + ids.append((child_stage_id, i, j)) + futures.append(f) + elif concurrency == 1 or part == 0: + ids.append((child_stage_id, i, 0)) + futures.append(lst) + return ids, futures + + # if we are using disk-based shuffle, wait until the child stages to finish + # writing the shuffle files to disk first. + if not use_ray_shuffle: + ray.get([f for _, lst in child_outputs for f in lst]) + + # schedule the actual execution workers + plan_bytes = raysql.serialize_execution_plan(stage.get_execution_plan()) + futures = [] + opt = {} + opt["resources"] = {"worker": 1e-3} + if use_ray_shuffle: + opt["num_returns"] = output_partitions_count + for part in range(concurrency): + ids, inputs = _get_worker_inputs(part) + futures.append( + execute_query_partition.options(**opt).remote( + stage_id, plan_bytes, part, ids, *inputs + ) + ) + + return stage_id, futures + + +@ray.remote +def execute_query_partition( + stage_id: int, + plan_bytes: bytes, + part: int, + input_partition_ids: list[tuple[int, int, int]], + *input_partitions: list[pa.RecordBatch], +) -> Iterable[pa.RecordBatch]: + start_time = time.time() + plan = raysql.deserialize_execution_plan(plan_bytes) + # print( + # "Worker executing plan {} partition #{} with shuffle inputs {}".format( + # plan.display(), + # part, + # input_partition_ids, + # ) + # ) + partitions = [ + (s, j, p) for (s, _, j), p in zip(input_partition_ids, input_partitions) + ] + # This is delegating to DataFusion for execution, but this would be a good place + # to plug in other execution engines by translating the plan into another engine's plan + # (perhaps via Substrait, once DataFusion supports converting a physical plan to Substrait) + ret = raysql.execute_partition(plan, part, partitions) + duration = time.time() - start_time + event = { + "cat": f"{stage_id}-{part}", + "name": f"{stage_id}-{part}", + "pid": ray.util.get_node_ip_address(), + "tid": os.getpid(), + "ts": int(start_time * 1_000_000), + "dur": int(duration * 1_000_000), + "ph": "X", + } + print(json.dumps(event), end=",") + return ret[0] if len(ret) == 1 else ret + + +class RaySqlContext: + def __init__(self, num_workers: int = 1, use_ray_shuffle: bool = False): + self.ctx = Context(num_workers, use_ray_shuffle) + self.num_workers = num_workers + self.use_ray_shuffle = use_ray_shuffle + + def register_csv(self, table_name: str, path: str, has_header: bool): + self.ctx.register_csv(table_name, path, has_header) + + def register_parquet(self, table_name: str, path: str): + self.ctx.register_parquet(table_name, path) + + def register_data_lake(self, table_name: str, paths: List[str]): + self.ctx.register_datalake_table(table_name, paths) + + def sql(self, sql: str) -> pa.RecordBatch: + # TODO we should parse sql and inspect the plan rather than + # perform a string comparison here + sql_str = sql.lower() + if "create view" in sql_str or "drop view" in sql_str: + self.ctx.sql(sql) + return [] + + graph = self.ctx.plan(sql) + final_stage_id = graph.get_final_query_stage().id() + if self.use_ray_shuffle: + partitions = schedule_execution(graph, final_stage_id, True) + else: + # serialize the query stages and store in Ray object store + query_stages = [ + raysql.serialize_execution_plan( + graph.get_query_stage(i).get_execution_plan() + ) + for i in range(final_stage_id + 1) + ] + # schedule execution + future = execute_query_stage.remote( + query_stages, + final_stage_id, + self.use_ray_shuffle, + ) + _, partitions = ray.get(future) + # assert len(partitions) == 1, len(partitions) + result_set = ray.get(partitions[0]) + return result_set diff --git a/raysql/main.py b/raysql/main.py new file mode 100644 index 0000000..e9fe382 --- /dev/null +++ b/raysql/main.py @@ -0,0 +1,124 @@ +# 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 time +import os + +from pyarrow import csv as pacsv +import ray +from raysql import RaySqlContext + +NUM_CPUS_PER_WORKER = 8 + +SF = 10 +DATA_DIR = f"/mnt/data0/tpch/sf{SF}-parquet" +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +QUERIES_DIR = os.path.join(SCRIPT_DIR, f"../sqlbench-h/queries/sf={SF}") +RESULTS_DIR = f"results-sf{SF}" +TRUTH_DIR = ( + "/home/ubuntu/raysort/ray-sql/sqlbench-runners/spark/{RESULTS_DIR}/{RESULTS_DIR}" +) + + +def setup_context(use_ray_shuffle: bool, num_workers: int = 2) -> RaySqlContext: + print(f"Using {num_workers} workers") + ctx = RaySqlContext(num_workers, use_ray_shuffle) + for table in [ + "customer", + "lineitem", + "nation", + "orders", + "part", + "partsupp", + "region", + "supplier", + ]: + ctx.register_parquet(table, f"{DATA_DIR}/{table}.parquet") + return ctx + + +def load_query(n: int) -> str: + with open(f"{QUERIES_DIR}/q{n}.sql") as fin: + return fin.read() + + +def tpch_query(ctx: RaySqlContext, q: int = 1): + sql = load_query(q) + result_set = ctx.sql(sql) + return result_set + + +def tpch_timing( + ctx: RaySqlContext, + q: int = 1, + print_result: bool = False, + write_result: bool = False, +): + sql = load_query(q) + start = time.perf_counter() + result = ctx.sql(sql) + end = time.perf_counter() + if print_result: + print("Result:", result) + if isinstance(result, list): + for r in result: + print(r.to_pandas()) + else: + print(result.to_pandas()) + if write_result: + opt = pacsv.WriteOptions(quoting_style="none") + if isinstance(result, list): + for r in result: + pacsv.write_csv(r, f"{RESULTS_DIR}/q{q}.csv", write_options=opt) + else: + pacsv.write_csv(result, f"{RESULTS_DIR}/q{q}.csv", write_options=opt) + return end - start + + +def compare(q: int): + ctx = setup_context(False) + result_set_truth = tpch_query(ctx, q) + + ctx = setup_context(True) + result_set_ray = tpch_query(ctx, q) + + assert result_set_truth == result_set_ray, ( + q, + result_set_truth, + result_set_ray, + ) + + +def tpch_bench(): + ray.init("auto") + num_workers = int(ray.cluster_resources().get("worker", 1)) * NUM_CPUS_PER_WORKER + use_ray_shuffle = False + ctx = setup_context(use_ray_shuffle, num_workers) + # t = tpch_timing(ctx, 11, print_result=True) + # print(f"query,{t},{use_ray_shuffle},{num_workers}") + # return + run_id = time.strftime("%Y-%m-%d-%H-%M-%S") + with open(f"results-sf{SF}-{run_id}.csv", "w") as fout: + for i in range(1, 22 + 1): + if i == 15: + continue + result = tpch_timing(ctx, i, write_result=True) + print(f"query,{i},{result}") + print(f"query,{i},{result}", file=fout, flush=True) + + +tpch_bench() diff --git a/raysql/ray_utils.py b/raysql/ray_utils.py new file mode 100644 index 0000000..6c1eda4 --- /dev/null +++ b/raysql/ray_utils.py @@ -0,0 +1,31 @@ +# 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 ray + + +def node_aff(node_id: ray.NodeID, *, soft: bool = False) -> dict: + return { + "scheduling_strategy": ray.util.scheduling_strategies.NodeAffinitySchedulingStrategy( + node_id=node_id, + soft=soft, + ) + } + + +def current_node_aff() -> dict: + return node_aff(ray.get_runtime_context().get_node_id()) diff --git a/raysql/tests/test_context.py b/raysql/tests/test_context.py new file mode 100644 index 0000000..e1cc97c --- /dev/null +++ b/raysql/tests/test_context.py @@ -0,0 +1,24 @@ +# 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 pytest +from raysql import Context + +def test(): + ctx = Context(1, False) + ctx.register_csv('tips', 'examples/tips.csv', True) + ctx.plan("SELECT * FROM tips") diff --git a/requirements-in.txt b/requirements-in.txt new file mode 100644 index 0000000..b0ba082 --- /dev/null +++ b/requirements-in.txt @@ -0,0 +1,11 @@ +black +flake8 +isort +maturin[patchelf] +mypy +numpy +pyarrow +pytest +ray==2.3.0 +toml +importlib_metadata; python_version < "3.8" diff --git a/src/context.rs b/src/context.rs new file mode 100644 index 0000000..fbfccfb --- /dev/null +++ b/src/context.rs @@ -0,0 +1,261 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::planner::{make_execution_graph, PyExecutionGraph}; +use crate::shuffle::{RayShuffleReaderExec, ShuffleCodec}; +use crate::utils::wait_for_future; +use datafusion::arrow::pyarrow::FromPyArrow; +use datafusion::arrow::pyarrow::ToPyArrow; +use datafusion::arrow::record_batch::RecordBatch; +use datafusion::config::Extensions; +use datafusion::error::{DataFusionError, Result}; +use datafusion::execution::context::TaskContext; +use datafusion::execution::disk_manager::DiskManagerConfig; +use datafusion::execution::memory_pool::FairSpillPool; +use datafusion::execution::options::ReadOptions; +use datafusion::execution::runtime_env::RuntimeEnv; +use datafusion::physical_plan::{displayable, ExecutionPlan}; +use datafusion::prelude::*; +use datafusion_proto::bytes::{ + physical_plan_from_bytes_with_extension_codec, physical_plan_to_bytes_with_extension_codec, +}; +use datafusion_python::physical_plan::PyExecutionPlan; +use futures::StreamExt; +use pyo3::prelude::*; +use pyo3::types::{PyList, PyLong, PyTuple}; +use std::collections::HashMap; +use std::sync::Arc; +use tokio::runtime::Runtime; +use tokio::task::JoinHandle; + +type PyResultSet = Vec; + +#[pyclass(name = "Context", module = "raysql", subclass)] +pub struct PyContext { + pub(crate) ctx: SessionContext, + use_ray_shuffle: bool, +} + +#[pymethods] +impl PyContext { + #[new] + pub fn new(target_partitions: usize, use_ray_shuffle: bool) -> Result { + let config = SessionConfig::default() + .with_target_partitions(target_partitions) + .with_batch_size(16 * 1024) + .with_repartition_aggregations(true) + .with_repartition_windows(true) + .with_repartition_joins(true) + .with_parquet_pruning(true); + + let mem_pool_size = 1024 * 1024 * 1024; + let runtime_config = datafusion::execution::runtime_env::RuntimeConfig::new() + .with_memory_pool(Arc::new(FairSpillPool::new(mem_pool_size))) + .with_disk_manager(DiskManagerConfig::new_specified(vec!["/tmp".into()])); + let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); + let ctx = SessionContext::with_config_rt(config, runtime); + Ok(Self { + ctx, + use_ray_shuffle, + }) + } + + pub fn register_csv( + &self, + name: &str, + path: &str, + has_header: bool, + py: Python, + ) -> PyResult<()> { + let options = CsvReadOptions::default().has_header(has_header); + wait_for_future(py, self.ctx.register_csv(name, path, options))?; + Ok(()) + } + + pub fn register_parquet(&self, name: &str, path: &str, py: Python) -> PyResult<()> { + let options = ParquetReadOptions::default(); + wait_for_future(py, self.ctx.register_parquet(name, path, options))?; + Ok(()) + } + + pub fn register_datalake_table(&self, name: &str, path: Vec<&str>, py: Python) -> PyResult<()> { + // let options = ParquetReadOptions::default(); + // let listing_options = options.to_listing_options(&self.ctx.state().config()); + // wait_for_future(py, self.ctx.register_listing_table(name, path, listing_options, None, None))?; + // Ok(()) + unimplemented!() + } + + /// Execute SQL directly against the DataFusion context. Useful for statements + /// such as "create view" or "drop view" + pub fn sql(&self, sql: &str, py: Python) -> PyResult<()> { + println!("Executing {}", sql); + let _df = wait_for_future(py, self.ctx.sql(sql))?; + Ok(()) + } + + /// Plan a distributed SELECT query for executing against the Ray workers + pub fn plan(&self, sql: &str, py: Python) -> PyResult { + println!("Planning {}", sql); + let df = wait_for_future(py, self.ctx.sql(sql))?; + let plan = wait_for_future(py, df.create_physical_plan())?; + + let graph = make_execution_graph(plan.clone(), self.use_ray_shuffle)?; + + // debug logging + let mut stages = graph.query_stages.values().collect::>(); + stages.sort_by_key(|s| s.id); + for stage in stages { + println!( + "Query stage #{}:\n{}", + stage.id, + displayable(stage.plan.as_ref()).indent(false) + ); + } + + Ok(PyExecutionGraph::new(graph)) + } + + /// Execute a partition of a query plan. This will typically be executing a shuffle write and write the results to disk + pub fn execute_partition( + &self, + plan: PyExecutionPlan, + part: usize, + inputs: PyObject, + py: Python, + ) -> PyResultSet { + execute_partition(plan, part, inputs, py) + } +} + +#[pyfunction] +pub fn execute_partition( + plan: PyExecutionPlan, + part: usize, + inputs: PyObject, + py: Python, +) -> PyResultSet { + _execute_partition(plan, part, inputs) + .unwrap() + .into_iter() + .map(|batch| batch.to_pyarrow(py).unwrap()) // TODO(@lsf) handle error + .collect() +} + +// TODO(@lsf) change this to use pickle +#[pyfunction] +pub fn serialize_execution_plan(plan: PyExecutionPlan) -> PyResult> { + let codec = ShuffleCodec {}; + Ok(physical_plan_to_bytes_with_extension_codec(plan.plan, &codec)?.to_vec()) +} + +#[pyfunction] +pub fn deserialize_execution_plan(bytes: Vec) -> PyResult { + let ctx = SessionContext::new(); + let codec = ShuffleCodec {}; + Ok(PyExecutionPlan::new( + physical_plan_from_bytes_with_extension_codec(&bytes, &ctx, &codec)?, + )) +} + +/// Iterate down an ExecutionPlan and set the input objects for RayShuffleReaderExec. +fn _set_inputs_for_ray_shuffle_reader( + plan: Arc, + part: usize, + input_partitions: &PyList, +) -> Result<()> { + if let Some(reader_exec) = plan.as_any().downcast_ref::() { + let exec_stage_id = reader_exec.stage_id; + // iterate over inputs, wrap in PyBytes and set as input objects + for item in input_partitions.iter() { + let pytuple = item + .downcast::() + .map_err(|e| DataFusionError::Execution(format!("{}", e)))?; + let stage_id = pytuple + .get_item(0) + .map_err(|e| DataFusionError::Execution(format!("{}", e)))? + .downcast::() + .map_err(|e| DataFusionError::Execution(format!("{}", e)))? + .extract::() + .map_err(|e| DataFusionError::Execution(format!("{}", e)))?; + if stage_id != exec_stage_id { + continue; + } + let part = pytuple + .get_item(1) + .map_err(|e| DataFusionError::Execution(format!("{}", e)))? + .downcast::() + .map_err(|e| DataFusionError::Execution(format!("{}", e)))? + .extract::() + .map_err(|e| DataFusionError::Execution(format!("{}", e)))?; + let batch = RecordBatch::from_pyarrow( + pytuple + .get_item(2) + .map_err(|e| DataFusionError::Execution(format!("{}", e)))?, + ) + .map_err(|e| DataFusionError::Execution(format!("{}", e)))?; + reader_exec.add_input_partition(part, batch)?; + } + } else { + for child in plan.children() { + _set_inputs_for_ray_shuffle_reader(child, part, input_partitions)?; + } + } + Ok(()) +} + +/// Execute a partition of a query plan. This will typically be executing a shuffle write and +/// write the results to disk, except for the final query stage, which will return the data. +/// inputs is a list of tuples of (stage_id, partition_id, bytes) for each input partition. +fn _execute_partition( + plan: PyExecutionPlan, + part: usize, + inputs: PyObject, +) -> Result> { + let ctx = Arc::new(TaskContext::try_new( + "task_id".to_string(), + "session_id".to_string(), + HashMap::new(), + HashMap::new(), + HashMap::new(), + Arc::new(RuntimeEnv::default()), + Extensions::default(), + )?); + Python::with_gil(|py| { + let input_partitions = inputs + .as_ref(py) + .downcast::() + .map_err(|e| DataFusionError::Execution(format!("{}", e)))?; + _set_inputs_for_ray_shuffle_reader(plan.plan.clone(), part, &input_partitions) + })?; + + // create a Tokio runtime to run the async code + let rt = Runtime::new().unwrap(); + + let fut: JoinHandle>> = rt.spawn(async move { + let mut stream = plan.plan.execute(part, ctx)?; + let mut results = vec![]; + while let Some(result) = stream.next().await { + results.push(result?); + } + Ok(results) + }); + + // block and wait on future + let results = rt.block_on(fut).unwrap()?; + Ok(results) +} diff --git a/src/lib.rs b/src/lib.rs new file mode 100644 index 0000000..df53b52 --- /dev/null +++ b/src/lib.rs @@ -0,0 +1,43 @@ +// 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. + +extern crate core; + +use pyo3::prelude::*; + +mod proto; +use crate::context::{deserialize_execution_plan, execute_partition, serialize_execution_plan}; +pub use proto::generated::protobuf; + +pub mod context; +pub mod planner; +pub mod query_stage; +pub mod shuffle; +pub mod utils; + +/// A Python module implemented in Rust. +#[pymodule] +fn _raysql_internal(_py: Python, m: &PyModule) -> PyResult<()> { + // register classes that can be created directly from Python code + m.add_class::()?; + m.add_class::()?; + m.add_class::()?; + m.add_function(wrap_pyfunction!(execute_partition, m)?)?; + m.add_function(wrap_pyfunction!(serialize_execution_plan, m)?)?; + m.add_function(wrap_pyfunction!(deserialize_execution_plan, m)?)?; + Ok(()) +} diff --git a/src/planner.rs b/src/planner.rs new file mode 100644 index 0000000..30b5b34 --- /dev/null +++ b/src/planner.rs @@ -0,0 +1,439 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::query_stage::PyQueryStage; +use crate::query_stage::QueryStage; +use crate::shuffle::{RayShuffleReaderExec, RayShuffleWriterExec}; +use crate::shuffle::{ShuffleReaderExec, ShuffleWriterExec}; +use datafusion::error::Result; +use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion::physical_plan::repartition::RepartitionExec; +use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion::physical_plan::{displayable, Partitioning}; +use datafusion::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; +use log::debug; +use pyo3::prelude::*; +use std::collections::HashMap; +use std::sync::atomic::{AtomicUsize, Ordering}; +use std::sync::Arc; +use uuid::Uuid; + +#[pyclass(name = "ExecutionGraph", module = "raysql", subclass)] +pub struct PyExecutionGraph { + pub graph: ExecutionGraph, +} + +impl PyExecutionGraph { + pub fn new(graph: ExecutionGraph) -> Self { + Self { graph } + } +} + +#[pymethods] +impl PyExecutionGraph { + /// Get a list of stages sorted by id + pub fn get_query_stages(&self) -> Vec { + let mut stages = vec![]; + let max_id = self.graph.get_final_query_stage().id; + for id in 0..=max_id { + stages.push(PyQueryStage::from_rust( + self.graph.query_stages.get(&id).unwrap().clone(), + )); + } + stages + } + + pub fn get_query_stage(&self, id: usize) -> PyResult { + if let Some(stage) = self.graph.query_stages.get(&id) { + Ok(PyQueryStage::from_rust(stage.clone())) + } else { + todo!() + } + } + + pub fn get_final_query_stage(&self) -> PyQueryStage { + PyQueryStage::from_rust(self.graph.get_final_query_stage()) + } +} + +#[derive(Debug)] +pub struct ExecutionGraph { + /// Query stages by id + pub query_stages: HashMap>, + id_generator: AtomicUsize, +} + +impl Default for ExecutionGraph { + fn default() -> Self { + Self::new() + } +} + +impl ExecutionGraph { + pub fn new() -> Self { + Self { + query_stages: HashMap::new(), + id_generator: AtomicUsize::new(0), + } + } + + fn add_query_stage(&mut self, stage_id: usize, plan: Arc) -> usize { + let query_stage = QueryStage::new(stage_id, plan); + self.query_stages.insert(stage_id, Arc::new(query_stage)); + stage_id + } + + fn get_final_query_stage(&self) -> Arc { + // the final query stage is always the last to be created and + // therefore has the highest id + let mut max_id = 0; + for k in self.query_stages.keys() { + if *k > max_id { + max_id = *k; + } + } + self.query_stages.get(&max_id).unwrap().clone() + } + + fn next_id(&self) -> usize { + self.id_generator.fetch_add(1, Ordering::Relaxed) + } +} + +pub fn make_execution_graph( + plan: Arc, + use_ray_shuffle: bool, +) -> Result { + let mut graph = ExecutionGraph::new(); + let root = generate_query_stages(plan, &mut graph, use_ray_shuffle)?; + // We force the final stage to produce a single partition to return + // to the driver. This might not suit ETL workloads. + if root.output_partitioning().partition_count() > 1 { + let root = Arc::new(CoalescePartitionsExec::new(root)); + graph.add_query_stage(graph.next_id(), root); + } else { + graph.add_query_stage(graph.next_id(), root); + } + Ok(graph) +} + +/// Convert a physical query plan into a distributed physical query plan by breaking the query +/// into query stages based on changes in partitioning. +fn generate_query_stages( + plan: Arc, + graph: &mut ExecutionGraph, + use_ray_shuffle: bool, +) -> Result> { + // recurse down first + let new_children: Vec> = plan + .children() + .iter() + .map(|x| generate_query_stages(x.clone(), graph, use_ray_shuffle)) + .collect::>>()?; + let plan = with_new_children_if_necessary(plan, new_children)?.into(); + + debug!("plan = {}", displayable(plan.as_ref()).one_line()); + debug!("output_part = {:?}", plan.output_partitioning()); + + let new_plan = if let Some(repart) = plan.as_any().downcast_ref::() { + match repart.partitioning() { + &Partitioning::UnknownPartitioning(_) | &Partitioning::RoundRobinBatch(_) => { + // just remove these + Ok(repart.children()[0].clone()) + } + partitioning_scheme => create_shuffle_exchange( + plan.children()[0].clone(), + graph, + partitioning_scheme.clone(), + use_ray_shuffle, + ), + } + } else if plan + .as_any() + .downcast_ref::() + .is_some() + { + let coalesce_input = plan.children()[0].clone(); + let partitioning_scheme = coalesce_input.output_partitioning(); + let new_input = + create_shuffle_exchange(coalesce_input, graph, partitioning_scheme, use_ray_shuffle)?; + with_new_children_if_necessary(plan, vec![new_input]).map(|p| p.into()) + } else if plan + .as_any() + .downcast_ref::() + .is_some() + { + let partitioned_sort_plan = plan.children()[0].clone(); + let partitioning_scheme = partitioned_sort_plan.output_partitioning(); + let new_input = create_shuffle_exchange( + partitioned_sort_plan, + graph, + partitioning_scheme, + use_ray_shuffle, + )?; + with_new_children_if_necessary(plan, vec![new_input]).map(|p| p.into()) + } else { + Ok(plan) + }?; + + debug!("new_plan = {}", displayable(new_plan.as_ref()).one_line()); + debug!( + "new_output_part = {:?}\n\n-------------------------\n\n", + new_plan.output_partitioning() + ); + + Ok(new_plan) +} + +/// Create a shuffle exchange. +/// +/// The plan is wrapped in a ShuffleWriteExec and added as a new query plan in the execution graph +/// and a ShuffleReaderExec is returned to replace the plan. +fn create_shuffle_exchange( + plan: Arc, + graph: &mut ExecutionGraph, + partitioning_scheme: Partitioning, + use_ray_shuffle: bool, +) -> Result> { + // introduce shuffle to produce one output partition + let stage_id = graph.next_id(); + + // create temp dir for stage shuffle files + let temp_dir = create_temp_dir(stage_id)?; + + let shuffle_writer_input = plan.clone(); + let shuffle_writer: Arc = if use_ray_shuffle { + Arc::new(RayShuffleWriterExec::new( + stage_id, + shuffle_writer_input, + partitioning_scheme.clone(), + )) + } else { + Arc::new(ShuffleWriterExec::new( + stage_id, + shuffle_writer_input, + partitioning_scheme.clone(), + &temp_dir, + )) + }; + + debug!( + "Created shuffle writer with output partitioning {:?}", + shuffle_writer.output_partitioning() + ); + + let stage_id = graph.add_query_stage(stage_id, shuffle_writer); + // replace the plan with a shuffle reader + if use_ray_shuffle { + Ok(Arc::new(RayShuffleReaderExec::new( + stage_id, + plan.schema(), + partitioning_scheme, + ))) + } else { + Ok(Arc::new(ShuffleReaderExec::new( + stage_id, + plan.schema(), + partitioning_scheme, + &temp_dir, + ))) + } +} + +fn create_temp_dir(stage_id: usize) -> Result { + let uuid = Uuid::new_v4(); + let temp_dir = format!("/tmp/ray-sql-{uuid}-stage-{stage_id}"); + debug!("Creating temp shuffle dir: {temp_dir}"); + std::fs::create_dir(&temp_dir)?; + Ok(temp_dir) +} + +#[cfg(test)] +mod test { + use super::*; + use datafusion::physical_plan::displayable; + use datafusion::prelude::{ParquetReadOptions, SessionConfig, SessionContext}; + use std::fs; + use std::path::Path; + + #[tokio::test] + async fn test_q1() -> Result<()> { + do_test(1).await + } + + #[tokio::test] + async fn test_q2() -> Result<()> { + do_test(2).await + } + + #[tokio::test] + async fn test_q3() -> Result<()> { + do_test(3).await + } + + #[tokio::test] + async fn test_q4() -> Result<()> { + do_test(4).await + } + + #[tokio::test] + async fn test_q5() -> Result<()> { + do_test(5).await + } + + #[tokio::test] + async fn test_q6() -> Result<()> { + do_test(6).await + } + + #[tokio::test] + async fn test_q7() -> Result<()> { + do_test(7).await + } + + #[tokio::test] + async fn test_q8() -> Result<()> { + do_test(8).await + } + + #[tokio::test] + async fn test_q9() -> Result<()> { + do_test(9).await + } + + #[tokio::test] + async fn test_q10() -> Result<()> { + do_test(10).await + } + + #[tokio::test] + async fn test_q11() -> Result<()> { + do_test(11).await + } + + #[tokio::test] + async fn test_q12() -> Result<()> { + do_test(12).await + } + + #[tokio::test] + async fn test_q13() -> Result<()> { + do_test(13).await + } + + #[tokio::test] + async fn test_q14() -> Result<()> { + do_test(14).await + } + + #[ignore] + #[tokio::test] + async fn test_q15() -> Result<()> { + do_test(15).await + } + + #[tokio::test] + async fn test_q16() -> Result<()> { + do_test(16).await + } + + #[tokio::test] + async fn test_q17() -> Result<()> { + do_test(17).await + } + + #[tokio::test] + async fn test_q18() -> Result<()> { + do_test(18).await + } + + #[tokio::test] + async fn test_q19() -> Result<()> { + do_test(19).await + } + + #[tokio::test] + async fn test_q20() -> Result<()> { + do_test(20).await + } + + #[tokio::test] + async fn test_q21() -> Result<()> { + do_test(21).await + } + + #[tokio::test] + async fn test_q22() -> Result<()> { + 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(()); + } + let file = format!("testdata/queries/q{n}.sql"); + let sql = fs::read_to_string(&file)?; + let config = SessionConfig::new().with_target_partitions(4); + let ctx = SessionContext::with_config(config); + let tables = &[ + "customer", "lineitem", "nation", "orders", "part", "partsupp", "region", "supplier", + ]; + for table in tables { + ctx.register_parquet( + table, + &format!("{data_path}/{table}.parquet"), + ParquetReadOptions::default(), + ) + .await?; + } + let mut output = String::new(); + + let df = ctx.sql(&sql).await?; + + let plan = df.clone().into_optimized_plan()?; + output.push_str(&format!( + "DataFusion Logical Plan\n=======================\n\n{}\n\n", + plan.display_indent() + )); + + let plan = df.create_physical_plan().await?; + output.push_str(&format!( + "DataFusion Physical Plan\n========================\n\n{}\n", + displayable(plan.as_ref()).indent(false) + )); + + output.push_str("RaySQL Plan\n===========\n\n"); + let graph = make_execution_graph(plan, false)?; + for id in 0..=graph.get_final_query_stage().id { + let query_stage = graph.query_stages.get(&id).unwrap(); + output.push_str(&format!( + "Query Stage #{id} ({} -> {}):\n{}\n", + query_stage.get_input_partition_count(), + query_stage.get_output_partition_count(), + displayable(query_stage.plan.as_ref()).indent(false) + )); + } + let expected_file = format!("testdata/expected-plans/q{n}.txt"); + if !Path::new(&expected_file).exists() { + fs::write(&expected_file, &output)?; + } + let expected_plan = fs::read_to_string(&expected_file)?; + assert_eq!(expected_plan, output); + Ok(()) + } +} diff --git a/src/proto/datafusion.proto b/src/proto/datafusion.proto new file mode 100644 index 0000000..8a9a8c0 --- /dev/null +++ b/src/proto/datafusion.proto @@ -0,0 +1,1347 @@ +/* + * 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. + */ + +syntax = "proto3"; + +package datafusion; + +option java_multiple_files = true; +option java_package = "org.apache.arrow.datafusion.protobuf"; +option java_outer_classname = "DatafusionProto"; + +message ColumnRelation { + string relation = 1; +} + +message Column { + string name = 1; + ColumnRelation relation = 2; +} + +message DfField{ + Field field = 1; + ColumnRelation qualifier = 2; +} + +message DfSchema { + repeated DfField columns = 1; + map metadata = 2; +} + +// logical plan +// LogicalPlan is a nested type +message LogicalPlanNode { + oneof LogicalPlanType { + ListingTableScanNode listing_scan = 1; + ProjectionNode projection = 3; + SelectionNode selection = 4; + LimitNode limit = 5; + AggregateNode aggregate = 6; + JoinNode join = 7; + SortNode sort = 8; + RepartitionNode repartition = 9; + EmptyRelationNode empty_relation = 10; + CreateExternalTableNode create_external_table = 11; + ExplainNode explain = 12; + WindowNode window = 13; + AnalyzeNode analyze = 14; + CrossJoinNode cross_join = 15; + ValuesNode values = 16; + LogicalExtensionNode extension = 17; + CreateCatalogSchemaNode create_catalog_schema = 18; + UnionNode union = 19; + CreateCatalogNode create_catalog = 20; + SubqueryAliasNode subquery_alias = 21; + CreateViewNode create_view = 22; + DistinctNode distinct = 23; + ViewTableScanNode view_scan = 24; + CustomTableScanNode custom_scan = 25; + PrepareNode prepare = 26; + } +} + +message LogicalExtensionNode { + bytes node = 1; + repeated LogicalPlanNode inputs = 2; +} + +message ProjectionColumns { + repeated string columns = 1; +} + +message CsvFormat { + bool has_header = 1; + string delimiter = 2; +} + +message ParquetFormat { + // Used to be bool enable_pruning = 1; + reserved 1; +} + +message AvroFormat {} + +message ListingTableScanNode { + string table_name = 1; + repeated string paths = 2; + string file_extension = 3; + ProjectionColumns projection = 4; + Schema schema = 5; + repeated LogicalExprNode filters = 6; + repeated string table_partition_cols = 7; + bool collect_stat = 8; + uint32 target_partitions = 9; + oneof FileFormatType { + CsvFormat csv = 10; + ParquetFormat parquet = 11; + AvroFormat avro = 12; + } + repeated LogicalExprNode file_sort_order = 13; +} + +message ViewTableScanNode { + string table_name = 1; + LogicalPlanNode input = 2; + Schema schema = 3; + ProjectionColumns projection = 4; + string definition = 5; +} + +// Logical Plan to Scan a CustomTableProvider registered at runtime +message CustomTableScanNode { + string table_name = 1; + ProjectionColumns projection = 2; + Schema schema = 3; + repeated LogicalExprNode filters = 4; + bytes custom_table_data = 5; +} + +message ProjectionNode { + LogicalPlanNode input = 1; + repeated LogicalExprNode expr = 2; + oneof optional_alias { + string alias = 3; + } +} + +message SelectionNode { + LogicalPlanNode input = 1; + LogicalExprNode expr = 2; +} + +message SortNode { + LogicalPlanNode input = 1; + repeated LogicalExprNode expr = 2; + // Maximum number of highest/lowest rows to fetch; negative means no limit + int64 fetch = 3; +} + +message RepartitionNode { + LogicalPlanNode input = 1; + oneof partition_method { + uint64 round_robin = 2; + HashRepartition hash = 3; + } +} + +message HashRepartition { + repeated LogicalExprNode hash_expr = 1; + uint64 partition_count = 2; +} + +message EmptyRelationNode { + bool produce_one_row = 1; +} + +message CreateExternalTableNode { + reserved 1; // was string name + OwnedTableReference name = 12; + string location = 2; + string file_type = 3; + bool has_header = 4; + DfSchema schema = 5; + repeated string table_partition_cols = 6; + bool if_not_exists = 7; + string delimiter = 8; + string definition = 9; + string file_compression_type = 10; + map options = 11; +} + +message PrepareNode { + string name = 1; + repeated ArrowType data_types = 2; + LogicalPlanNode input = 3; +} + +message CreateCatalogSchemaNode { + string schema_name = 1; + bool if_not_exists = 2; + DfSchema schema = 3; +} + +message CreateCatalogNode { + string catalog_name = 1; + bool if_not_exists = 2; + DfSchema schema = 3; +} + +message CreateViewNode { + reserved 1; // was string name + OwnedTableReference name = 5; + LogicalPlanNode input = 2; + bool or_replace = 3; + string definition = 4; +} + +// a node containing data for defining values list. unlike in SQL where it's two dimensional, here +// the list is flattened, and with the field n_cols it can be parsed and partitioned into rows +message ValuesNode { + uint64 n_cols = 1; + repeated LogicalExprNode values_list = 2; +} + +message AnalyzeNode { + LogicalPlanNode input = 1; + bool verbose = 2; +} + +message ExplainNode { + LogicalPlanNode input = 1; + bool verbose = 2; +} + +message AggregateNode { + LogicalPlanNode input = 1; + repeated LogicalExprNode group_expr = 2; + repeated LogicalExprNode aggr_expr = 3; +} + +message WindowNode { + LogicalPlanNode input = 1; + repeated LogicalExprNode window_expr = 2; +} + +enum JoinType { + INNER = 0; + LEFT = 1; + RIGHT = 2; + FULL = 3; + LEFTSEMI = 4; + LEFTANTI = 5; + RIGHTSEMI = 6; + RIGHTANTI = 7; +} + +enum JoinConstraint { + ON = 0; + USING = 1; +} + +message JoinNode { + LogicalPlanNode left = 1; + LogicalPlanNode right = 2; + JoinType join_type = 3; + JoinConstraint join_constraint = 4; + repeated LogicalExprNode left_join_key = 5; + repeated LogicalExprNode right_join_key = 6; + bool null_equals_null = 7; + LogicalExprNode filter = 8; +} + +message DistinctNode { + LogicalPlanNode input = 1; +} + +message UnionNode { + repeated LogicalPlanNode inputs = 1; +} + +message CrossJoinNode { + LogicalPlanNode left = 1; + LogicalPlanNode right = 2; +} + +message LimitNode { + LogicalPlanNode input = 1; + // The number of rows to skip before fetch; non-positive means don't skip any + int64 skip = 2; + // Maximum number of rows to fetch; negative means no limit + int64 fetch = 3; +} + +message SelectionExecNode { + LogicalExprNode expr = 1; +} + +message SubqueryAliasNode { + LogicalPlanNode input = 1; + string alias = 2; +} + +// logical expressions +message LogicalExprNode { + oneof ExprType { + // column references + Column column = 1; + + // alias + AliasNode alias = 2; + + ScalarValue literal = 3; + + // binary expressions + BinaryExprNode binary_expr = 4; + + // aggregate expressions + AggregateExprNode aggregate_expr = 5; + + // null checks + IsNull is_null_expr = 6; + IsNotNull is_not_null_expr = 7; + Not not_expr = 8; + + BetweenNode between = 9; + CaseNode case_ = 10; + CastNode cast = 11; + SortExprNode sort = 12; + NegativeNode negative = 13; + InListNode in_list = 14; + bool wildcard = 15; + ScalarFunctionNode scalar_function = 16; + TryCastNode try_cast = 17; + + // window expressions + WindowExprNode window_expr = 18; + + // AggregateUDF expressions + AggregateUDFExprNode aggregate_udf_expr = 19; + + // Scalar UDF expressions + ScalarUDFExprNode scalar_udf_expr = 20; + + GetIndexedField get_indexed_field = 21; + + GroupingSetNode grouping_set = 22; + + CubeNode cube = 23; + + RollupNode rollup = 24; + + IsTrue is_true = 25; + IsFalse is_false = 26; + IsUnknown is_unknown = 27; + IsNotTrue is_not_true = 28; + IsNotFalse is_not_false = 29; + IsNotUnknown is_not_unknown = 30; + LikeNode like = 31; + ILikeNode ilike = 32; + SimilarToNode similar_to = 33; + + PlaceholderNode placeholder = 34; + + } +} + +message PlaceholderNode { + string id = 1; + ArrowType data_type = 2; +} + +message LogicalExprList { + repeated LogicalExprNode expr = 1; +} + +message GroupingSetNode { + repeated LogicalExprList expr = 1; +} + +message CubeNode { + repeated LogicalExprNode expr = 1; +} + +message RollupNode { + repeated LogicalExprNode expr = 1; +} + + + +message GetIndexedField { + LogicalExprNode expr = 1; + ScalarValue key = 2; +} + +message IsNull { + LogicalExprNode expr = 1; +} + +message IsNotNull { + LogicalExprNode expr = 1; +} + +message IsTrue { + LogicalExprNode expr = 1; +} + +message IsFalse { + LogicalExprNode expr = 1; +} + +message IsUnknown { + LogicalExprNode expr = 1; +} + +message IsNotTrue { + LogicalExprNode expr = 1; +} + +message IsNotFalse { + LogicalExprNode expr = 1; +} + +message IsNotUnknown { + LogicalExprNode expr = 1; +} + +message Not { + LogicalExprNode expr = 1; +} + +message AliasNode { + LogicalExprNode expr = 1; + string alias = 2; +} + +message BinaryExprNode { + // Represents the operands from the left inner most expression + // to the right outer most expression where each of them are chained + // with the operator 'op'. + repeated LogicalExprNode operands = 1; + string op = 3; +} + +message NegativeNode { + LogicalExprNode expr = 1; +} + +message InListNode { + LogicalExprNode expr = 1; + repeated LogicalExprNode list = 2; + bool negated = 3; +} + +enum ScalarFunction { + Abs = 0; + Acos = 1; + Asin = 2; + Atan = 3; + Ascii = 4; + Ceil = 5; + Cos = 6; + Digest = 7; + Exp = 8; + Floor = 9; + Ln = 10; + Log = 11; + Log10 = 12; + Log2 = 13; + Round = 14; + Signum = 15; + Sin = 16; + Sqrt = 17; + Tan = 18; + Trunc = 19; + Array = 20; + RegexpMatch = 21; + BitLength = 22; + Btrim = 23; + CharacterLength = 24; + Chr = 25; + Concat = 26; + ConcatWithSeparator = 27; + DatePart = 28; + DateTrunc = 29; + InitCap = 30; + Left = 31; + Lpad = 32; + Lower = 33; + Ltrim = 34; + MD5 = 35; + NullIf = 36; + OctetLength = 37; + Random = 38; + RegexpReplace = 39; + Repeat = 40; + Replace = 41; + Reverse = 42; + Right = 43; + Rpad = 44; + Rtrim = 45; + SHA224 = 46; + SHA256 = 47; + SHA384 = 48; + SHA512 = 49; + SplitPart = 50; + StartsWith = 51; + Strpos = 52; + Substr = 53; + ToHex = 54; + ToTimestamp = 55; + ToTimestampMillis = 56; + ToTimestampMicros = 57; + ToTimestampSeconds = 58; + Now = 59; + Translate = 60; + Trim = 61; + Upper = 62; + Coalesce = 63; + Power = 64; + StructFun = 65; + FromUnixtime = 66; + Atan2 = 67; + DateBin = 68; + ArrowTypeof = 69; + CurrentDate = 70; + CurrentTime = 71; + Uuid = 72; +} + +message ScalarFunctionNode { + ScalarFunction fun = 1; + repeated LogicalExprNode args = 2; +} + +enum AggregateFunction { + MIN = 0; + MAX = 1; + SUM = 2; + AVG = 3; + COUNT = 4; + APPROX_DISTINCT = 5; + ARRAY_AGG = 6; + VARIANCE = 7; + VARIANCE_POP = 8; + COVARIANCE = 9; + COVARIANCE_POP = 10; + STDDEV = 11; + STDDEV_POP = 12; + CORRELATION = 13; + APPROX_PERCENTILE_CONT = 14; + APPROX_MEDIAN = 15; + APPROX_PERCENTILE_CONT_WITH_WEIGHT = 16; + GROUPING = 17; + MEDIAN = 18; +} + +message AggregateExprNode { + AggregateFunction aggr_function = 1; + repeated LogicalExprNode expr = 2; + bool distinct = 3; + LogicalExprNode filter = 4; +} + +message AggregateUDFExprNode { + string fun_name = 1; + repeated LogicalExprNode args = 2; + LogicalExprNode filter = 3; +} + +message ScalarUDFExprNode { + string fun_name = 1; + repeated LogicalExprNode args = 2; +} + +enum BuiltInWindowFunction { + ROW_NUMBER = 0; + RANK = 1; + DENSE_RANK = 2; + PERCENT_RANK = 3; + CUME_DIST = 4; + NTILE = 5; + LAG = 6; + LEAD = 7; + FIRST_VALUE = 8; + LAST_VALUE = 9; + NTH_VALUE = 10; +} + +message WindowExprNode { + oneof window_function { + AggregateFunction aggr_function = 1; + BuiltInWindowFunction built_in_function = 2; + // udaf = 3 + } + LogicalExprNode expr = 4; + repeated LogicalExprNode partition_by = 5; + repeated LogicalExprNode order_by = 6; + // repeated LogicalExprNode filter = 7; + WindowFrame window_frame = 8; +} + +message BetweenNode { + LogicalExprNode expr = 1; + bool negated = 2; + LogicalExprNode low = 3; + LogicalExprNode high = 4; +} + +message LikeNode { + bool negated = 1; + LogicalExprNode expr = 2; + LogicalExprNode pattern = 3; + string escape_char = 4; +} + +message ILikeNode { + bool negated = 1; + LogicalExprNode expr = 2; + LogicalExprNode pattern = 3; + string escape_char = 4; +} + +message SimilarToNode { + bool negated = 1; + LogicalExprNode expr = 2; + LogicalExprNode pattern = 3; + string escape_char = 4; +} + +message CaseNode { + LogicalExprNode expr = 1; + repeated WhenThen when_then_expr = 2; + LogicalExprNode else_expr = 3; +} + +message WhenThen { + LogicalExprNode when_expr = 1; + LogicalExprNode then_expr = 2; +} + +message CastNode { + LogicalExprNode expr = 1; + ArrowType arrow_type = 2; +} + +message TryCastNode { + LogicalExprNode expr = 1; + ArrowType arrow_type = 2; +} + +message SortExprNode { + LogicalExprNode expr = 1; + bool asc = 2; + bool nulls_first = 3; +} + +enum WindowFrameUnits { + ROWS = 0; + RANGE = 1; + GROUPS = 2; +} + +message WindowFrame { + WindowFrameUnits window_frame_units = 1; + WindowFrameBound start_bound = 2; + // "optional" keyword is stable in protoc 3.15 but prost is still on 3.14 (see https://github.com/tokio-rs/prost/issues/430 and https://github.com/tokio-rs/prost/pull/455) + // this syntax is ugly but is binary compatible with the "optional" keyword (see https://stackoverflow.com/questions/42622015/how-to-define-an-optional-field-in-protobuf-3) + oneof end_bound { + WindowFrameBound bound = 3; + } +} + +enum WindowFrameBoundType { + CURRENT_ROW = 0; + PRECEDING = 1; + FOLLOWING = 2; +} + +message WindowFrameBound { + WindowFrameBoundType window_frame_bound_type = 1; + ScalarValue bound_value = 2; +} + +/////////////////////////////////////////////////////////////////////////////////////////////////// +// Arrow Data Types +/////////////////////////////////////////////////////////////////////////////////////////////////// + +message Schema { + repeated Field columns = 1; +} + +message Field { + // name of the field + string name = 1; + ArrowType arrow_type = 2; + bool nullable = 3; + // for complex data types like structs, unions + repeated Field children = 4; +} + +message FixedSizeBinary{ + int32 length = 1; +} + +message Timestamp{ + TimeUnit time_unit = 1; + string timezone = 2; +} + +enum DateUnit{ + Day = 0; + DateMillisecond = 1; +} + +enum TimeUnit{ + Second = 0; + Millisecond = 1; + Microsecond = 2; + Nanosecond = 3; +} + +enum IntervalUnit{ + YearMonth = 0; + DayTime = 1; + MonthDayNano = 2; +} + +message Decimal{ + reserved 1, 2; + uint32 precision = 3; + int32 scale = 4; +} + +message List{ + Field field_type = 1; +} + +message FixedSizeList{ + Field field_type = 1; + int32 list_size = 2; +} + +message Dictionary{ + ArrowType key = 1; + ArrowType value = 2; +} + +message Struct{ + repeated Field sub_field_types = 1; +} + +enum UnionMode{ + sparse = 0; + dense = 1; +} + +message Union{ + repeated Field union_types = 1; + UnionMode union_mode = 2; + repeated int32 type_ids = 3; +} + +message ScalarListValue{ + // encode null explicitly to distinguish a list with a null value + // from a list with no values) + bool is_null = 3; + Field field = 1; + repeated ScalarValue values = 2; +} + +message ScalarTime32Value { + oneof value { + int32 time32_second_value = 1; + int32 time32_millisecond_value = 2; + }; +} + +message ScalarTime64Value { + oneof value { + int64 time64_microsecond_value = 1; + int64 time64_nanosecond_value = 2; + }; +} + +message ScalarTimestampValue { + oneof value { + int64 time_microsecond_value = 1; + int64 time_nanosecond_value = 2; + int64 time_second_value = 3; + int64 time_millisecond_value = 4; + }; + string timezone = 5; +} + +message ScalarDictionaryValue { + ArrowType index_type = 1; + ScalarValue value = 2; +} + +message IntervalMonthDayNanoValue { + int32 months = 1; + int32 days = 2; + int64 nanos = 3; +} + +message StructValue { + // Note that a null struct value must have one or more fields, so we + // encode a null StructValue as one witth an empty field_values + // list. + repeated ScalarValue field_values = 2; + repeated Field fields = 3; +} + +message ScalarFixedSizeBinary{ + bytes values = 1; + int32 length = 2; +} + +message ScalarValue{ + // was PrimitiveScalarType null_value = 19; + reserved 19; + + oneof value { + // was PrimitiveScalarType null_value = 19; + // Null value of any type + ArrowType null_value = 33; + + bool bool_value = 1; + string utf8_value = 2; + string large_utf8_value = 3; + int32 int8_value = 4; + int32 int16_value = 5; + int32 int32_value = 6; + int64 int64_value = 7; + uint32 uint8_value = 8; + uint32 uint16_value = 9; + uint32 uint32_value = 10; + uint64 uint64_value = 11; + float float32_value = 12; + double float64_value = 13; + // Literal Date32 value always has a unit of day + int32 date_32_value = 14; + ScalarTime32Value time32_value = 15; + ScalarListValue list_value = 17; + //WAS: ScalarType null_list_value = 18; + + Decimal128 decimal128_value = 20; + int64 date_64_value = 21; + int32 interval_yearmonth_value = 24; + int64 interval_daytime_value = 25; + ScalarTimestampValue timestamp_value = 26; + ScalarDictionaryValue dictionary_value = 27; + bytes binary_value = 28; + bytes large_binary_value = 29; + ScalarTime64Value time64_value = 30; + IntervalMonthDayNanoValue interval_month_day_nano = 31; + StructValue struct_value = 32; + ScalarFixedSizeBinary fixed_size_binary_value = 34; + } +} + +message Decimal128{ + bytes value = 1; + int64 p = 2; + int64 s = 3; +} + +// Serialized data type +message ArrowType{ + oneof arrow_type_enum { + EmptyMessage NONE = 1; // arrow::Type::NA + EmptyMessage BOOL = 2; // arrow::Type::BOOL + EmptyMessage UINT8 = 3; // arrow::Type::UINT8 + EmptyMessage INT8 = 4; // arrow::Type::INT8 + EmptyMessage UINT16 = 5; // represents arrow::Type fields in src/arrow/type.h + EmptyMessage INT16 = 6; + EmptyMessage UINT32 = 7; + EmptyMessage INT32 = 8; + EmptyMessage UINT64 = 9; + EmptyMessage INT64 = 10 ; + EmptyMessage FLOAT16 = 11 ; + EmptyMessage FLOAT32 = 12 ; + EmptyMessage FLOAT64 = 13 ; + EmptyMessage UTF8 = 14 ; + EmptyMessage LARGE_UTF8 = 32; + EmptyMessage BINARY = 15 ; + int32 FIXED_SIZE_BINARY = 16 ; + EmptyMessage LARGE_BINARY = 31; + EmptyMessage DATE32 = 17 ; + EmptyMessage DATE64 = 18 ; + TimeUnit DURATION = 19; + Timestamp TIMESTAMP = 20 ; + TimeUnit TIME32 = 21 ; + TimeUnit TIME64 = 22 ; + IntervalUnit INTERVAL = 23 ; + Decimal DECIMAL = 24 ; + List LIST = 25; + List LARGE_LIST = 26; + FixedSizeList FIXED_SIZE_LIST = 27; + Struct STRUCT = 28; + Union UNION = 29; + Dictionary DICTIONARY = 30; + } +} + +//Useful for representing an empty enum variant in rust +// E.G. enum example{One, Two(i32)} +// maps to +// message example{ +// oneof{ +// EmptyMessage One = 1; +// i32 Two = 2; +// } +//} +message EmptyMessage{} + +message OptimizedLogicalPlanType { + string optimizer_name = 1; +} + +message OptimizedPhysicalPlanType { + string optimizer_name = 1; +} + +message PlanType { + oneof plan_type_enum { + EmptyMessage InitialLogicalPlan = 1; + OptimizedLogicalPlanType OptimizedLogicalPlan = 2; + EmptyMessage FinalLogicalPlan = 3; + EmptyMessage InitialPhysicalPlan = 4; + OptimizedPhysicalPlanType OptimizedPhysicalPlan = 5; + EmptyMessage FinalPhysicalPlan = 6; + } +} + +message StringifiedPlan { + PlanType plan_type = 1; + string plan = 2; +} + +message BareTableReference { + string table = 1; +} + +message PartialTableReference { + string schema = 1; + string table = 2; +} + +message FullTableReference { + string catalog = 1; + string schema = 2; + string table = 3; +} + +message OwnedTableReference { + oneof table_reference_enum { + BareTableReference bare = 1; + PartialTableReference partial = 2; + FullTableReference full = 3; + } +} + +///////////////////////////////////////////////////////////////////////////////////////////////// + +// PhysicalPlanNode is a nested type +message PhysicalPlanNode { + oneof PhysicalPlanType { + ParquetScanExecNode parquet_scan = 1; + CsvScanExecNode csv_scan = 2; + EmptyExecNode empty = 3; + ProjectionExecNode projection = 4; + GlobalLimitExecNode global_limit = 6; + LocalLimitExecNode local_limit = 7; + AggregateExecNode aggregate = 8; + HashJoinExecNode hash_join = 9; + SortExecNode sort = 10; + CoalesceBatchesExecNode coalesce_batches = 11; + FilterExecNode filter = 12; + CoalescePartitionsExecNode merge = 13; + RepartitionExecNode repartition = 14; + WindowAggExecNode window = 15; + CrossJoinExecNode cross_join = 16; + AvroScanExecNode avro_scan = 17; + PhysicalExtensionNode extension = 18; + UnionExecNode union = 19; + ExplainExecNode explain = 20; + SortPreservingMergeExecNode sort_preserving_merge = 21; + } +} + +message PhysicalExtensionNode { + bytes node = 1; + repeated PhysicalPlanNode inputs = 2; +} + +// physical expressions +message PhysicalExprNode { + oneof ExprType { + // column references + PhysicalColumn column = 1; + + ScalarValue literal = 2; + + // binary expressions + PhysicalBinaryExprNode binary_expr = 3; + + // aggregate expressions + PhysicalAggregateExprNode aggregate_expr = 4; + + // null checks + PhysicalIsNull is_null_expr = 5; + PhysicalIsNotNull is_not_null_expr = 6; + PhysicalNot not_expr = 7; + + PhysicalCaseNode case_ = 8; + PhysicalCastNode cast = 9; + PhysicalSortExprNode sort = 10; + PhysicalNegativeNode negative = 11; + PhysicalInListNode in_list = 12; + PhysicalScalarFunctionNode scalar_function = 13; + PhysicalTryCastNode try_cast = 14; + + // window expressions + PhysicalWindowExprNode window_expr = 15; + + PhysicalScalarUdfNode scalar_udf = 16; + + PhysicalDateTimeIntervalExprNode date_time_interval_expr = 17; + + PhysicalLikeExprNode like_expr = 18; + } +} + +message PhysicalScalarUdfNode { + string name = 1; + repeated PhysicalExprNode args = 2; + ArrowType return_type = 4; +} + +message PhysicalAggregateExprNode { + AggregateFunction aggr_function = 1; + repeated PhysicalExprNode expr = 2; + bool distinct = 3; +} + +message PhysicalWindowExprNode { + oneof window_function { + AggregateFunction aggr_function = 1; + BuiltInWindowFunction built_in_function = 2; + // udaf = 3 + } + PhysicalExprNode expr = 4; +} + +message PhysicalIsNull { + PhysicalExprNode expr = 1; +} + +message PhysicalIsNotNull { + PhysicalExprNode expr = 1; +} + +message PhysicalNot { + PhysicalExprNode expr = 1; +} + +message PhysicalAliasNode { + PhysicalExprNode expr = 1; + string alias = 2; +} + +message PhysicalBinaryExprNode { + PhysicalExprNode l = 1; + PhysicalExprNode r = 2; + string op = 3; +} + +message PhysicalDateTimeIntervalExprNode { + PhysicalExprNode l = 1; + PhysicalExprNode r = 2; + string op = 3; +} + +message PhysicalLikeExprNode { + bool negated = 1; + bool case_insensitive = 2; + PhysicalExprNode expr = 3; + PhysicalExprNode pattern = 4; +} + +message PhysicalSortExprNode { + PhysicalExprNode expr = 1; + bool asc = 2; + bool nulls_first = 3; +} + +message PhysicalWhenThen { + PhysicalExprNode when_expr = 1; + PhysicalExprNode then_expr = 2; +} + +message PhysicalInListNode { + PhysicalExprNode expr = 1; + repeated PhysicalExprNode list = 2; + bool negated = 3; +} + +message PhysicalCaseNode { + PhysicalExprNode expr = 1; + repeated PhysicalWhenThen when_then_expr = 2; + PhysicalExprNode else_expr = 3; +} + +message PhysicalScalarFunctionNode { + string name = 1; + ScalarFunction fun = 2; + repeated PhysicalExprNode args = 3; + ArrowType return_type = 4; +} + +message PhysicalTryCastNode { + PhysicalExprNode expr = 1; + ArrowType arrow_type = 2; +} + +message PhysicalCastNode { + PhysicalExprNode expr = 1; + ArrowType arrow_type = 2; +} + +message PhysicalNegativeNode { + PhysicalExprNode expr = 1; +} + +message FilterExecNode { + PhysicalPlanNode input = 1; + PhysicalExprNode expr = 2; +} + +message FileGroup { + repeated PartitionedFile files = 1; +} + +message ScanLimit { + // wrap into a message to make it optional + uint32 limit = 1; +} + +message FileScanExecConf { + // Was repeated ConfigOption options = 10; + reserved 10; + + repeated FileGroup file_groups = 1; + Schema schema = 2; + repeated uint32 projection = 4; + ScanLimit limit = 5; + Statistics statistics = 6; + repeated string table_partition_cols = 7; + string object_store_url = 8; + repeated PhysicalSortExprNode output_ordering = 9; +} + +message ParquetScanExecNode { + FileScanExecConf base_conf = 1; + LogicalExprNode pruning_predicate = 2; +} + +message CsvScanExecNode { + FileScanExecConf base_conf = 1; + bool has_header = 2; + string delimiter = 3; +} + +message AvroScanExecNode { + FileScanExecConf base_conf = 1; +} + +enum PartitionMode { + COLLECT_LEFT = 0; + PARTITIONED = 1; + AUTO = 2; +} + +message HashJoinExecNode { + PhysicalPlanNode left = 1; + PhysicalPlanNode right = 2; + repeated JoinOn on = 3; + JoinType join_type = 4; + PartitionMode partition_mode = 6; + bool null_equals_null = 7; + JoinFilter filter = 8; +} + +message UnionExecNode { + repeated PhysicalPlanNode inputs = 1; +} + +message ExplainExecNode { + Schema schema = 1; + repeated StringifiedPlan stringified_plans = 2; + bool verbose = 3; +} + +message CrossJoinExecNode { + PhysicalPlanNode left = 1; + PhysicalPlanNode right = 2; +} + +message PhysicalColumn { + string name = 1; + uint32 index = 2; +} + +message JoinOn { + PhysicalColumn left = 1; + PhysicalColumn right = 2; +} + +message EmptyExecNode { + bool produce_one_row = 1; + Schema schema = 2; +} + +message ProjectionExecNode { + PhysicalPlanNode input = 1; + repeated PhysicalExprNode expr = 2; + repeated string expr_name = 3; +} + +enum AggregateMode { + PARTIAL = 0; + FINAL = 1; + FINAL_PARTITIONED = 2; +} + +message WindowAggExecNode { + PhysicalPlanNode input = 1; + repeated PhysicalExprNode window_expr = 2; + repeated string window_expr_name = 3; + Schema input_schema = 4; +} + +message AggregateExecNode { + repeated PhysicalExprNode group_expr = 1; + repeated PhysicalExprNode aggr_expr = 2; + AggregateMode mode = 3; + PhysicalPlanNode input = 4; + repeated string group_expr_name = 5; + repeated string aggr_expr_name = 6; + // we need the input schema to the partial aggregate to pass to the final aggregate + Schema input_schema = 7; + repeated PhysicalExprNode null_expr = 8; + repeated bool groups = 9; +} + +message GlobalLimitExecNode { + PhysicalPlanNode input = 1; + // The number of rows to skip before fetch + uint32 skip = 2; + // Maximum number of rows to fetch; negative means no limit + int64 fetch = 3; +} + +message LocalLimitExecNode { + PhysicalPlanNode input = 1; + uint32 fetch = 2; +} + +message SortExecNode { + PhysicalPlanNode input = 1; + repeated PhysicalExprNode expr = 2; + // Maximum number of highest/lowest rows to fetch; negative means no limit + int64 fetch = 3; +} + +message SortPreservingMergeExecNode { + PhysicalPlanNode input = 1; + repeated PhysicalExprNode expr = 2; +} + +message CoalesceBatchesExecNode { + PhysicalPlanNode input = 1; + uint32 target_batch_size = 2; +} + +message CoalescePartitionsExecNode { + PhysicalPlanNode input = 1; +} + +message PhysicalHashRepartition { + repeated PhysicalExprNode hash_expr = 1; + uint64 partition_count = 2; +} + +message RepartitionExecNode{ + PhysicalPlanNode input = 1; + oneof partition_method { + uint64 round_robin = 2; + PhysicalHashRepartition hash = 3; + uint64 unknown = 4; + } +} + +message JoinFilter{ + PhysicalExprNode expression = 1; + repeated ColumnIndex column_indices = 2; + Schema schema = 3; +} + +message ColumnIndex{ + uint32 index = 1; + JoinSide side = 2; +} + +enum JoinSide{ + LEFT_SIDE = 0; + RIGHT_SIDE = 1; +} + +message PartitionedFile { + string path = 1; + uint64 size = 2; + uint64 last_modified_ns = 3; + repeated ScalarValue partition_values = 4; + FileRange range = 5; +} + +message FileRange { + int64 start = 1; + int64 end = 2; +} + +message PartitionStats { + int64 num_rows = 1; + int64 num_batches = 2; + int64 num_bytes = 3; + repeated ColumnStats column_stats = 4; +} + +message Statistics { + int64 num_rows = 1; + int64 total_byte_size = 2; + repeated ColumnStats column_stats = 3; + bool is_exact = 4; +} + +message ColumnStats { + ScalarValue min_value = 1; + ScalarValue max_value = 2; + uint32 null_count = 3; + uint32 distinct_count = 4; +} \ No newline at end of file diff --git a/src/proto/generated/mod.rs b/src/proto/generated/mod.rs new file mode 100644 index 0000000..833afac --- /dev/null +++ b/src/proto/generated/mod.rs @@ -0,0 +1,18 @@ +// 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. + +pub mod protobuf; diff --git a/src/proto/generated/protobuf.rs b/src/proto/generated/protobuf.rs new file mode 100644 index 0000000..c31d7ac --- /dev/null +++ b/src/proto/generated/protobuf.rs @@ -0,0 +1,87 @@ +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RaySqlExecNode { + #[prost(oneof = "ray_sql_exec_node::PlanType", tags = "1, 2, 3, 4")] + pub plan_type: ::core::option::Option, +} +/// Nested message and enum types in `RaySqlExecNode`. +pub mod ray_sql_exec_node { + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum PlanType { + #[prost(message, tag = "1")] + ShuffleReader(super::ShuffleReaderExecNode), + #[prost(message, tag = "2")] + ShuffleWriter(super::ShuffleWriterExecNode), + #[prost(message, tag = "3")] + RayShuffleReader(super::RayShuffleReaderExecNode), + #[prost(message, tag = "4")] + RayShuffleWriter(super::RayShuffleWriterExecNode), + } +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ShuffleReaderExecNode { + /// stage to read from + #[prost(uint32, tag = "1")] + pub stage_id: u32, + /// schema of the shuffle stage + #[prost(message, optional, tag = "2")] + pub schema: ::core::option::Option<::datafusion_proto::protobuf::Schema>, + /// this must match the output partitioning of the writer we are reading from + #[prost(message, optional, tag = "3")] + pub partitioning: ::core::option::Option< + ::datafusion_proto::protobuf::PhysicalHashRepartition, + >, + /// directory for shuffle files + #[prost(string, tag = "4")] + pub shuffle_dir: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ShuffleWriterExecNode { + /// stage that is writing the shuffle files + #[prost(uint32, tag = "1")] + pub stage_id: u32, + /// plan to execute + #[prost(message, optional, tag = "2")] + pub plan: ::core::option::Option<::datafusion_proto::protobuf::PhysicalPlanNode>, + /// output partitioning schema + #[prost(message, optional, tag = "3")] + pub partitioning: ::core::option::Option< + ::datafusion_proto::protobuf::PhysicalHashRepartition, + >, + /// directory for shuffle files + #[prost(string, tag = "4")] + pub shuffle_dir: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RayShuffleReaderExecNode { + /// stage to read from + #[prost(uint32, tag = "1")] + pub stage_id: u32, + /// schema of the shuffle stage + #[prost(message, optional, tag = "2")] + pub schema: ::core::option::Option<::datafusion_proto::protobuf::Schema>, + /// this must match the output partitioning of the writer we are reading from + #[prost(message, optional, tag = "3")] + pub partitioning: ::core::option::Option< + ::datafusion_proto::protobuf::PhysicalHashRepartition, + >, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RayShuffleWriterExecNode { + /// stage that is writing the shuffle files + #[prost(uint32, tag = "1")] + pub stage_id: u32, + /// plan to execute + #[prost(message, optional, tag = "2")] + pub plan: ::core::option::Option<::datafusion_proto::protobuf::PhysicalPlanNode>, + /// output partitioning schema + #[prost(message, optional, tag = "3")] + pub partitioning: ::core::option::Option< + ::datafusion_proto::protobuf::PhysicalHashRepartition, + >, +} diff --git a/src/proto/mod.rs b/src/proto/mod.rs new file mode 100644 index 0000000..8e28eb5 --- /dev/null +++ b/src/proto/mod.rs @@ -0,0 +1,18 @@ +// 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. + +pub mod generated; diff --git a/src/proto/raysql.proto b/src/proto/raysql.proto new file mode 100644 index 0000000..281e1ae --- /dev/null +++ b/src/proto/raysql.proto @@ -0,0 +1,58 @@ +syntax = "proto3"; + +package raysql.protobuf; + +option java_multiple_files = true; +option java_package = "raysql.protobuf"; +option java_outer_classname = "RaySqlProto"; + +import "datafusion.proto"; + +message RaySqlExecNode { + oneof PlanType { + ShuffleReaderExecNode shuffle_reader = 1; + ShuffleWriterExecNode shuffle_writer = 2; + RayShuffleReaderExecNode ray_shuffle_reader = 3; + RayShuffleWriterExecNode ray_shuffle_writer = 4; + } +} + +message ShuffleReaderExecNode { + // stage to read from + uint32 stage_id = 1; + // schema of the shuffle stage + datafusion.Schema schema = 2; + // this must match the output partitioning of the writer we are reading from + datafusion.PhysicalHashRepartition partitioning = 3; + // directory for shuffle files + string shuffle_dir = 4; +} + +message ShuffleWriterExecNode { + // stage that is writing the shuffle files + uint32 stage_id = 1; + // plan to execute + datafusion.PhysicalPlanNode plan = 2; + // output partitioning schema + datafusion.PhysicalHashRepartition partitioning = 3; + // directory for shuffle files + string shuffle_dir = 4; +} + +message RayShuffleReaderExecNode { + // stage to read from + uint32 stage_id = 1; + // schema of the shuffle stage + datafusion.Schema schema = 2; + // this must match the output partitioning of the writer we are reading from + datafusion.PhysicalHashRepartition partitioning = 3; +} + +message RayShuffleWriterExecNode { + // stage that is writing the shuffle files + uint32 stage_id = 1; + // plan to execute + datafusion.PhysicalPlanNode plan = 2; + // output partitioning schema + datafusion.PhysicalHashRepartition partitioning = 3; +} diff --git a/src/query_stage.rs b/src/query_stage.rs new file mode 100644 index 0000000..1bf2cbf --- /dev/null +++ b/src/query_stage.rs @@ -0,0 +1,121 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::shuffle::{RayShuffleReaderExec, ShuffleCodec, ShuffleReaderExec}; +use datafusion::error::Result; +use datafusion::physical_plan::{ExecutionPlan, Partitioning}; +use datafusion::prelude::SessionContext; +use datafusion_proto::bytes::physical_plan_from_bytes_with_extension_codec; +use datafusion_python::physical_plan::PyExecutionPlan; +use pyo3::prelude::*; +use std::sync::Arc; + +#[pyclass(name = "QueryStage", module = "raysql", subclass)] +pub struct PyQueryStage { + stage: Arc, +} + +impl PyQueryStage { + pub fn from_rust(stage: Arc) -> Self { + Self { stage } + } +} + +#[pymethods] +impl PyQueryStage { + #[new] + pub fn new(id: usize, bytes: Vec) -> Result { + let ctx = SessionContext::new(); + let codec = ShuffleCodec {}; + let plan = physical_plan_from_bytes_with_extension_codec(&bytes, &ctx, &codec)?; + Ok(PyQueryStage { + stage: Arc::new(QueryStage { id, plan }), + }) + } + + pub fn id(&self) -> usize { + self.stage.id + } + + pub fn get_execution_plan(&self) -> PyExecutionPlan { + PyExecutionPlan::new(self.stage.plan.clone()) + } + + pub fn get_child_stage_ids(&self) -> Vec { + self.stage.get_child_stage_ids() + } + + pub fn get_input_partition_count(&self) -> usize { + self.stage.get_input_partition_count() + } + + pub fn get_output_partition_count(&self) -> usize { + self.stage.get_output_partition_count() + } +} + +#[derive(Debug)] +pub struct QueryStage { + pub id: usize, + pub plan: Arc, +} + +fn _get_output_partition_count(plan: &dyn ExecutionPlan) -> usize { + // UnknownPartitioning and HashPartitioning with empty expressions will + // both return 1 partition. + match plan.output_partitioning() { + Partitioning::UnknownPartitioning(_) => 1, + Partitioning::Hash(expr, _) if expr.is_empty() => 1, + p => p.partition_count(), + } +} + +impl QueryStage { + pub fn new(id: usize, plan: Arc) -> Self { + Self { id, plan } + } + + pub fn get_child_stage_ids(&self) -> Vec { + let mut ids = vec![]; + collect_child_stage_ids(self.plan.as_ref(), &mut ids); + ids + } + + /// Get the input partition count. This is the same as the number of concurrent tasks + /// when we schedule this query stage for execution + pub fn get_input_partition_count(&self) -> usize { + self.plan.children()[0] + .output_partitioning() + .partition_count() + } + + pub fn get_output_partition_count(&self) -> usize { + _get_output_partition_count(self.plan.as_ref()) + } +} + +fn collect_child_stage_ids(plan: &dyn ExecutionPlan, ids: &mut Vec) { + if let Some(shuffle_reader) = plan.as_any().downcast_ref::() { + ids.push(shuffle_reader.stage_id); + } else if let Some(shuffle_reader) = plan.as_any().downcast_ref::() { + ids.push(shuffle_reader.stage_id); + } else { + for child_plan in plan.children() { + collect_child_stage_ids(child_plan.as_ref(), ids); + } + } +} diff --git a/src/shuffle/codec.rs b/src/shuffle/codec.rs new file mode 100644 index 0000000..ac0dfa5 --- /dev/null +++ b/src/shuffle/codec.rs @@ -0,0 +1,211 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::protobuf::ray_sql_exec_node::PlanType; +use crate::protobuf::{ + RayShuffleReaderExecNode, RayShuffleWriterExecNode, RaySqlExecNode, ShuffleReaderExecNode, + ShuffleWriterExecNode, +}; +use crate::shuffle::{ + RayShuffleReaderExec, RayShuffleWriterExec, ShuffleReaderExec, ShuffleWriterExec, +}; +use datafusion::arrow::datatypes::SchemaRef; +use datafusion::common::{DataFusionError, Result}; +use datafusion::execution::runtime_env::RuntimeEnv; +use datafusion::execution::FunctionRegistry; +use datafusion::logical_expr::{AggregateUDF, ScalarUDF, WindowUDF}; +use datafusion::physical_plan::{ExecutionPlan, Partitioning}; +use datafusion_proto::physical_plan::from_proto::parse_protobuf_hash_partitioning; +use datafusion_proto::physical_plan::AsExecutionPlan; +use datafusion_proto::physical_plan::PhysicalExtensionCodec; +use datafusion_proto::protobuf; +use datafusion_proto::protobuf::{PhysicalHashRepartition, PhysicalPlanNode}; +use prost::Message; +use std::collections::HashSet; +use std::sync::Arc; + +#[derive(Debug)] +pub struct ShuffleCodec {} + +impl PhysicalExtensionCodec for ShuffleCodec { + fn try_decode( + &self, + buf: &[u8], + _inputs: &[Arc], + registry: &dyn FunctionRegistry, + ) -> Result, DataFusionError> { + // decode bytes to protobuf struct + let node = RaySqlExecNode::decode(buf) + .map_err(|e| DataFusionError::Internal(format!("failed to decode plan: {e:?}")))?; + match node.plan_type { + Some(PlanType::ShuffleReader(reader)) => { + let schema = reader.schema.as_ref().unwrap(); + let schema: SchemaRef = Arc::new(schema.try_into().unwrap()); + let hash_part = parse_protobuf_hash_partitioning( + reader.partitioning.as_ref(), + registry, + &schema, + )?; + Ok(Arc::new(ShuffleReaderExec::new( + reader.stage_id as usize, + schema, + hash_part.unwrap(), + &reader.shuffle_dir, + ))) + } + Some(PlanType::ShuffleWriter(writer)) => { + let plan = writer.plan.unwrap().try_into_physical_plan( + registry, + &RuntimeEnv::default(), + self, + )?; + let hash_part = parse_protobuf_hash_partitioning( + writer.partitioning.as_ref(), + registry, + plan.schema().as_ref(), + )?; + Ok(Arc::new(ShuffleWriterExec::new( + writer.stage_id as usize, + plan, + hash_part.unwrap(), + &writer.shuffle_dir, + ))) + } + Some(PlanType::RayShuffleReader(reader)) => { + let schema = reader.schema.as_ref().unwrap(); + let schema: SchemaRef = Arc::new(schema.try_into().unwrap()); + let hash_part = parse_protobuf_hash_partitioning( + reader.partitioning.as_ref(), + registry, + &schema, + )?; + Ok(Arc::new(RayShuffleReaderExec::new( + reader.stage_id as usize, + schema, + hash_part.unwrap(), + ))) + } + Some(PlanType::RayShuffleWriter(writer)) => { + let plan = writer.plan.unwrap().try_into_physical_plan( + registry, + &RuntimeEnv::default(), + self, + )?; + let hash_part = parse_protobuf_hash_partitioning( + writer.partitioning.as_ref(), + registry, + plan.schema().as_ref(), + )?; + Ok(Arc::new(RayShuffleWriterExec::new( + writer.stage_id as usize, + plan, + hash_part.unwrap(), + ))) + } + _ => unreachable!(), + } + } + + fn try_encode( + &self, + node: Arc, + buf: &mut Vec, + ) -> Result<(), DataFusionError> { + let plan = if let Some(reader) = node.as_any().downcast_ref::() { + let schema: protobuf::Schema = reader.schema().try_into().unwrap(); + let partitioning = encode_partitioning_scheme(&reader.output_partitioning())?; + let reader = ShuffleReaderExecNode { + stage_id: reader.stage_id as u32, + schema: Some(schema), + partitioning: Some(partitioning), + shuffle_dir: reader.shuffle_dir.clone(), + }; + PlanType::ShuffleReader(reader) + } else if let Some(writer) = node.as_any().downcast_ref::() { + let plan = PhysicalPlanNode::try_from_physical_plan(writer.plan.clone(), self)?; + let partitioning = encode_partitioning_scheme(&writer.output_partitioning())?; + let writer = ShuffleWriterExecNode { + stage_id: writer.stage_id as u32, + plan: Some(plan), + partitioning: Some(partitioning), + shuffle_dir: writer.shuffle_dir.clone(), + }; + PlanType::ShuffleWriter(writer) + } else if let Some(reader) = node.as_any().downcast_ref::() { + let schema: protobuf::Schema = reader.schema().try_into().unwrap(); + let partitioning = encode_partitioning_scheme(&reader.output_partitioning())?; + let reader = RayShuffleReaderExecNode { + stage_id: reader.stage_id as u32, + schema: Some(schema), + partitioning: Some(partitioning), + }; + PlanType::RayShuffleReader(reader) + } else if let Some(writer) = node.as_any().downcast_ref::() { + let plan = PhysicalPlanNode::try_from_physical_plan(writer.plan.clone(), self)?; + let partitioning = encode_partitioning_scheme(&writer.output_partitioning())?; + let writer = RayShuffleWriterExecNode { + stage_id: writer.stage_id as u32, + plan: Some(plan), + partitioning: Some(partitioning), + }; + PlanType::RayShuffleWriter(writer) + } else { + unreachable!() + }; + plan.encode(buf); + Ok(()) + } +} + +fn encode_partitioning_scheme(partitioning: &Partitioning) -> Result { + match partitioning { + Partitioning::Hash(expr, partition_count) => Ok(protobuf::PhysicalHashRepartition { + hash_expr: expr + .iter() + .map(|expr| expr.clone().try_into()) + .collect::, DataFusionError>>()?, + partition_count: *partition_count as u64, + }), + Partitioning::UnknownPartitioning(n) => Ok(protobuf::PhysicalHashRepartition { + hash_expr: vec![], + partition_count: *n as u64, + }), + other => Err(DataFusionError::Plan(format!( + "Unsupported shuffle partitioning scheme: {other:?}" + ))), + } +} + +struct RaySqlFunctionRegistry {} + +impl FunctionRegistry for RaySqlFunctionRegistry { + fn udfs(&self) -> HashSet { + HashSet::new() + } + + fn udf(&self, name: &str) -> datafusion::common::Result> { + Err(DataFusionError::Plan(format!("Invalid UDF: {name}"))) + } + + fn udaf(&self, name: &str) -> datafusion::common::Result> { + Err(DataFusionError::Plan(format!("Invalid UDAF: {name}"))) + } + + fn udwf(&self, name: &str) -> datafusion::common::Result> { + Err(DataFusionError::Plan(format!("Invalid UDAWF: {name}"))) + } +} diff --git a/src/shuffle/mod.rs b/src/shuffle/mod.rs new file mode 100644 index 0000000..cfa3980 --- /dev/null +++ b/src/shuffle/mod.rs @@ -0,0 +1,27 @@ +// 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. + +mod codec; +mod ray_shuffle; +mod reader; +mod writer; + +pub use codec::ShuffleCodec; +pub use ray_shuffle::RayShuffleReaderExec; +pub use ray_shuffle::RayShuffleWriterExec; +pub use reader::ShuffleReaderExec; +pub use writer::ShuffleWriterExec; diff --git a/src/shuffle/ray_shuffle/mod.rs b/src/shuffle/ray_shuffle/mod.rs new file mode 100644 index 0000000..0a153b4 --- /dev/null +++ b/src/shuffle/ray_shuffle/mod.rs @@ -0,0 +1,95 @@ +// 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. + +mod reader; +mod writer; + +use arrow::record_batch::RecordBatch; +use datafusion::arrow; +use datafusion::arrow::datatypes::SchemaRef; +use datafusion::common::Result; +use datafusion::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; +use futures::Stream; +pub use reader::RayShuffleReaderExec; +use std::pin::Pin; +use std::task::{Context, Poll}; +use tokio::macros::support::thread_rng_n; +pub use writer::RayShuffleWriterExec; + +/// CombinedRecordBatchStream can be used to combine a Vec of SendableRecordBatchStreams into one +pub struct CombinedRecordBatchStream { + /// Schema wrapped by Arc + schema: SchemaRef, + /// Stream entries + entries: Vec, +} + +impl CombinedRecordBatchStream { + /// Create an CombinedRecordBatchStream + pub fn new(schema: SchemaRef, entries: Vec) -> Self { + Self { schema, entries } + } +} + +impl RecordBatchStream for CombinedRecordBatchStream { + fn schema(&self) -> SchemaRef { + self.schema.clone() + } +} + +impl Stream for CombinedRecordBatchStream { + type Item = Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + use Poll::*; + + let start = thread_rng_n(self.entries.len() as u32) as usize; + let mut idx = start; + + for _ in 0..self.entries.len() { + let stream = self.entries.get_mut(idx).unwrap(); + + match Pin::new(stream).poll_next(cx) { + Ready(Some(val)) => return Ready(Some(val)), + Ready(None) => { + // Remove the entry + self.entries.swap_remove(idx); + + // Check if this was the last entry, if so the cursor needs + // to wrap + if idx == self.entries.len() { + idx = 0; + } else if idx < start && start <= self.entries.len() { + // The stream being swapped into the current index has + // already been polled, so skip it. + idx = idx.wrapping_add(1) % self.entries.len(); + } + } + Pending => { + idx = idx.wrapping_add(1) % self.entries.len(); + } + } + } + + // If the map is empty, then the stream is complete. + if self.entries.is_empty() { + Ready(None) + } else { + Pending + } + } +} diff --git a/src/shuffle/ray_shuffle/reader.rs b/src/shuffle/ray_shuffle/reader.rs new file mode 100644 index 0000000..0b190ff --- /dev/null +++ b/src/shuffle/ray_shuffle/reader.rs @@ -0,0 +1,188 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::shuffle::ray_shuffle::CombinedRecordBatchStream; +use datafusion::arrow::datatypes::SchemaRef; +use datafusion::arrow::record_batch::RecordBatch; +use datafusion::common::Statistics; +use datafusion::error::{DataFusionError, Result}; +use datafusion::execution::context::TaskContext; +use datafusion::physical_expr::expressions::UnKnownColumn; +use datafusion::physical_expr::PhysicalSortExpr; +use datafusion::physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, + SendableRecordBatchStream, +}; +use futures::Stream; +use std::any::Any; +use std::collections::HashMap; +use std::fmt::Formatter; +use std::pin::Pin; +use std::sync::{Arc, RwLock}; +use std::task::{Context, Poll}; + +type PartitionId = usize; +type StageId = usize; + +#[derive(Debug)] +pub struct RayShuffleReaderExec { + /// Query stage to read from + pub stage_id: StageId, + /// The output schema of the query stage being read from + schema: SchemaRef, + /// Output partitioning + partitioning: Partitioning, + /// Input streams from Ray object store + input_partitions_map: RwLock>>, // TODO(@lsf) can we not use Rwlock? +} + +impl RayShuffleReaderExec { + pub fn new(stage_id: StageId, schema: SchemaRef, partitioning: Partitioning) -> Self { + let partitioning = match partitioning { + Partitioning::Hash(expr, n) if expr.is_empty() => Partitioning::UnknownPartitioning(n), + Partitioning::Hash(expr, n) => { + // workaround for DataFusion bug https://github.com/apache/arrow-datafusion/issues/5184 + Partitioning::Hash( + expr.into_iter() + .filter(|e| e.as_any().downcast_ref::().is_none()) + .collect(), + n, + ) + } + _ => partitioning, + }; + + Self { + stage_id, + schema, + partitioning, + input_partitions_map: RwLock::new(HashMap::new()), + } + } + + pub fn add_input_partition( + &self, + partition: PartitionId, + input_batch: RecordBatch, + ) -> Result<(), DataFusionError> { + let mut map = self.input_partitions_map.write().unwrap(); + let input_partitions = map.entry(partition).or_insert(vec![]); + input_partitions.push(input_batch); + Ok(()) + } +} + +impl ExecutionPlan for RayShuffleReaderExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn output_partitioning(&self) -> Partitioning { + self.partitioning.clone() + } + + fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + // TODO could be implemented in some cases + None + } + + fn children(&self) -> Vec> { + vec![] + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> datafusion::common::Result> { + Ok(self) + } + + fn execute( + &self, + partition: usize, + _context: Arc, + ) -> datafusion::common::Result { + let mut map = self.input_partitions_map.write().expect("got lock"); + let input_objects = map.remove(&partition).unwrap_or(vec![]); + println!( + "RayShuffleReaderExec[stage={}].execute(input_partition={partition}) with {} shuffle inputs", + self.stage_id, + input_objects.len(), + ); + let mut streams = vec![]; + for input in input_objects { + streams.push( + Box::pin(InMemoryShuffleStream::try_new(input)?) as SendableRecordBatchStream + ); + } + Ok(Box::pin(CombinedRecordBatchStream::new( + self.schema.clone(), + streams, + ))) + } + + fn statistics(&self) -> Result { + Ok(Statistics::new_unknown(&self.schema)) + } +} + +impl DisplayAs for RayShuffleReaderExec { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + write!( + f, + "RayShuffleReaderExec(stage_id={}, input_partitioning={:?})", + self.stage_id, self.partitioning + ) + } +} + +struct InMemoryShuffleStream { + batch: Arc, + read: bool, +} + +impl InMemoryShuffleStream { + fn try_new(batch: RecordBatch) -> Result { + Ok(Self { + batch: Arc::new(batch), + read: false, + }) + } +} + +impl Stream for InMemoryShuffleStream { + type Item = datafusion::error::Result; + + fn poll_next(mut self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { + Poll::Ready(if self.read { + None + } else { + self.read = true; + Some(Ok(self.batch.as_ref().clone())) + }) + } +} + +impl RecordBatchStream for InMemoryShuffleStream { + fn schema(&self) -> SchemaRef { + self.batch.schema() + } +} diff --git a/src/shuffle/ray_shuffle/writer.rs b/src/shuffle/ray_shuffle/writer.rs new file mode 100644 index 0000000..b33840d --- /dev/null +++ b/src/shuffle/ray_shuffle/writer.rs @@ -0,0 +1,238 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use datafusion::arrow::compute::concat_batches; +use datafusion::arrow::datatypes::SchemaRef; +use datafusion::arrow::record_batch::RecordBatch; +use datafusion::common::{Result, Statistics}; +use datafusion::error::DataFusionError; +use datafusion::execution::context::TaskContext; +use datafusion::physical_expr::expressions::UnKnownColumn; +use datafusion::physical_expr::PhysicalSortExpr; +use datafusion::physical_plan::common::batch_byte_size; +use datafusion::physical_plan::memory::MemoryStream; +use datafusion::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricBuilder}; +use datafusion::physical_plan::repartition::BatchPartitioner; +use datafusion::physical_plan::stream::RecordBatchStreamAdapter; +use datafusion::physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, +}; +use futures::StreamExt; +use futures::TryStreamExt; +use log::debug; +use std::any::Any; +use std::fmt::Formatter; +use std::sync::Arc; + +#[derive(Debug)] +pub struct RayShuffleWriterExec { + pub stage_id: usize, + /// The child execution plan + pub(crate) plan: Arc, + /// Output partitioning + partitioning: Partitioning, + /// Metrics + pub metrics: ExecutionPlanMetricsSet, +} + +impl RayShuffleWriterExec { + pub fn new(stage_id: usize, plan: Arc, partitioning: Partitioning) -> Self { + let partitioning = match partitioning { + Partitioning::Hash(expr, n) if expr.is_empty() => Partitioning::UnknownPartitioning(n), + Partitioning::Hash(expr, n) => { + // workaround for DataFusion bug https://github.com/apache/arrow-datafusion/issues/5184 + Partitioning::Hash( + expr.into_iter() + .filter(|e| e.as_any().downcast_ref::().is_none()) + .collect(), + n, + ) + } + _ => partitioning, + }; + + Self { + stage_id, + plan, + partitioning, + metrics: ExecutionPlanMetricsSet::new(), + } + } +} + +impl ExecutionPlan for RayShuffleWriterExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.plan.schema() + } + + fn output_partitioning(&self) -> Partitioning { + self.partitioning.clone() + } + + fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + // TODO in the case of a single partition of a sorted plan this could be implemented + None + } + + fn children(&self) -> Vec> { + vec![self.plan.clone()] + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + unimplemented!() + } + + fn execute( + &self, + input_partition: usize, + context: Arc, + ) -> Result { + println!( + "RayShuffleWriterExec[stage={}].execute(input_partition={input_partition})", + self.stage_id + ); + let mut stream = self.plan.execute(input_partition, context)?; + + let stage_id = self.stage_id; + let partitioning = self.output_partitioning(); + let partition_count = partitioning.partition_count(); + let repart_time = + MetricBuilder::new(&self.metrics).subset_time("repart_time", input_partition); + let schema = Arc::new(self.schema().as_ref().clone()); + + let results = async move { + // TODO(@lsf): why can't I reference self in here? + match &partitioning { + Partitioning::UnknownPartitioning(_) => { + let mut writer = InMemoryWriter::new(schema.clone()); + while let Some(result) = stream.next().await { + writer.write(result?)?; + } + println!( + "RayShuffleWriterExec[stage={}] Finished writing shuffle partition 0. Batches: {}. Rows: {}. Bytes: {}.", + stage_id, + writer.num_batches, + writer.num_rows, + writer.num_bytes + ); + MemoryStream::try_new(vec![writer.finish()?], schema, None) + } + Partitioning::Hash(_, _) => { + let mut writers: Vec = vec![]; + for _ in 0..partition_count { + writers.push(InMemoryWriter::new(schema.clone())); + } + + let mut partitioner = + BatchPartitioner::try_new(partitioning, repart_time.clone())?; + + let mut rows = 0; + + while let Some(result) = stream.next().await { + let input_batch = result?; + rows += input_batch.num_rows(); + partitioner.partition(input_batch, |output_partition, output_batch| { + writers[output_partition].write(output_batch) + })?; + } + let mut result_batches = vec![]; + for (i, w) in writers.iter_mut().enumerate() { + println!( + "RayShuffleWriterExec[stage={}] Finished writing shuffle partition {}. Batches: {}. Rows: {}. Bytes: {}.", + stage_id, + i, + w.num_batches, + w.num_rows, + w.num_bytes + ); + result_batches.push(w.finish()?); + } + debug!( + "RayShuffleWriterExec[stage={}] finished processing stream with {rows} rows", + stage_id + ); + MemoryStream::try_new(result_batches, schema, None) + } + _ => unimplemented!(), + } + }; + Ok(Box::pin(RecordBatchStreamAdapter::new( + self.schema(), + futures::stream::once(results).try_flatten(), + ))) + } + + fn statistics(&self) -> Result { + Ok(Statistics::new_unknown(&self.schema())) + } +} + +impl DisplayAs for RayShuffleWriterExec { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + write!( + f, + "RayShuffleWriterExec(stage_id={}, output_partitioning={:?})", + self.stage_id, self.partitioning + ) + } +} + +struct InMemoryWriter { + /// batches buffer + batches: Vec, + /// schema + schema: SchemaRef, + /// batches written + pub num_batches: u64, + /// rows written + pub num_rows: u64, + /// bytes written + pub num_bytes: u64, +} + +impl InMemoryWriter { + fn new(schema: SchemaRef) -> Self { + Self { + batches: vec![], + schema: schema, + num_batches: 0, + num_rows: 0, + num_bytes: 0, + } + } + + fn write(&mut self, batch: RecordBatch) -> Result<()> { + self.num_batches += 1; + self.num_rows += batch.num_rows() as u64; + self.num_bytes += batch_byte_size(&batch) as u64; + self.batches.push(batch); + Ok(()) + } + + fn finish(&self) -> Result { + // TODO(@lsf) Instead of concatenating the batches, return all RecordBatches from + // all partitions in one stream, then return an array of batch offsets. + concat_batches(&self.schema, &self.batches).map_err(DataFusionError::ArrowError) + } +} diff --git a/src/shuffle/reader.rs b/src/shuffle/reader.rs new file mode 100644 index 0000000..5c74b27 --- /dev/null +++ b/src/shuffle/reader.rs @@ -0,0 +1,185 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::shuffle::ray_shuffle::CombinedRecordBatchStream; +use datafusion::arrow::datatypes::SchemaRef; +use datafusion::arrow::ipc::reader::FileReader; +use datafusion::arrow::record_batch::RecordBatch; +use datafusion::common::Statistics; +use datafusion::error::{DataFusionError, Result}; +use datafusion::execution::context::TaskContext; +use datafusion::physical_expr::expressions::UnKnownColumn; +use datafusion::physical_expr::PhysicalSortExpr; +use datafusion::physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, + SendableRecordBatchStream, +}; +use futures::Stream; +use glob::glob; +use log::debug; +use std::any::Any; +use std::fmt::Formatter; +use std::fs::File; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; + +#[derive(Debug)] +pub struct ShuffleReaderExec { + /// Query stage to read from + pub stage_id: usize, + /// The output schema of the query stage being read from + schema: SchemaRef, + /// Output partitioning + partitioning: Partitioning, + /// Directory to read shuffle files from + pub shuffle_dir: String, +} + +impl ShuffleReaderExec { + pub fn new( + stage_id: usize, + schema: SchemaRef, + partitioning: Partitioning, + shuffle_dir: &str, + ) -> Self { + let partitioning = match partitioning { + Partitioning::Hash(expr, n) if expr.is_empty() => Partitioning::UnknownPartitioning(n), + Partitioning::Hash(expr, n) => { + // workaround for DataFusion bug https://github.com/apache/arrow-datafusion/issues/5184 + Partitioning::Hash( + expr.into_iter() + .filter(|e| e.as_any().downcast_ref::().is_none()) + .collect(), + n, + ) + } + _ => partitioning, + }; + + Self { + stage_id, + schema, + partitioning, + shuffle_dir: shuffle_dir.to_string(), + } + } +} + +impl ExecutionPlan for ShuffleReaderExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn output_partitioning(&self) -> Partitioning { + self.partitioning.clone() + } + + fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + // TODO could be implemented in some cases + None + } + + fn children(&self) -> Vec> { + vec![] + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> datafusion::common::Result> { + Ok(self) + } + + fn execute( + &self, + partition: usize, + _context: Arc, + ) -> datafusion::common::Result { + let pattern = format!( + "/{}/shuffle_{}_*_{partition}.arrow", + self.shuffle_dir, self.stage_id + ); + let mut streams: Vec = vec![]; + for entry in glob(&pattern).expect("Failed to read glob pattern") { + let file = entry.unwrap(); + debug!( + "ShuffleReaderExec partition {} reading from stage {} file {}", + partition, + self.stage_id, + file.display() + ); + let reader = FileReader::try_new(File::open(&file)?, None)?; + let stream = LocalShuffleStream::new(reader); + if self.schema != stream.schema() { + return Err(DataFusionError::Internal( + "Not all shuffle files have the same schema".to_string(), + )); + } + streams.push(Box::pin(stream)); + } + Ok(Box::pin(CombinedRecordBatchStream::new( + self.schema.clone(), + streams, + ))) + } + + fn statistics(&self) -> Result { + Ok(Statistics::new_unknown(&self.schema)) + } +} + +impl DisplayAs for ShuffleReaderExec { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + write!( + f, + "ShuffleReaderExec(stage_id={}, input_partitioning={:?})", + self.stage_id, self.partitioning + ) + } +} + +struct LocalShuffleStream { + reader: FileReader, +} + +impl LocalShuffleStream { + pub fn new(reader: FileReader) -> Self { + LocalShuffleStream { reader } + } +} + +impl Stream for LocalShuffleStream { + type Item = datafusion::error::Result; + + fn poll_next(mut self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { + if let Some(batch) = self.reader.next() { + return Poll::Ready(Some(batch.map_err(|e| e.into()))); + } + Poll::Ready(None) + } +} + +impl RecordBatchStream for LocalShuffleStream { + fn schema(&self) -> SchemaRef { + self.reader.schema() + } +} diff --git a/src/shuffle/writer.rs b/src/shuffle/writer.rs new file mode 100644 index 0000000..a2938bc --- /dev/null +++ b/src/shuffle/writer.rs @@ -0,0 +1,304 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use datafusion::arrow::array::Int32Array; +use datafusion::arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion::arrow::ipc::writer::FileWriter; +use datafusion::arrow::record_batch::RecordBatch; +use datafusion::arrow::util::pretty::pretty_format_batches; +use datafusion::common::{Result, Statistics}; +use datafusion::execution::context::TaskContext; +use datafusion::physical_expr::expressions::UnKnownColumn; +use datafusion::physical_expr::PhysicalSortExpr; +use datafusion::physical_plan::common::{batch_byte_size, IPCWriter}; +use datafusion::physical_plan::memory::MemoryStream; +use datafusion::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricBuilder}; +use datafusion::physical_plan::repartition::BatchPartitioner; +use datafusion::physical_plan::stream::RecordBatchStreamAdapter; +use datafusion::physical_plan::{ + metrics, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, + SendableRecordBatchStream, +}; +use datafusion_proto::protobuf::PartitionStats; +use futures::StreamExt; +use futures::TryStreamExt; +use log::debug; +use std::any::Any; +use std::fmt::Formatter; +use std::fs::File; +use std::path::Path; +use std::pin::Pin; +use std::sync::Arc; + +#[derive(Debug)] +pub struct ShuffleWriterExec { + pub stage_id: usize, + pub(crate) plan: Arc, + /// Output partitioning + partitioning: Partitioning, + /// Directory to write shuffle files from + pub shuffle_dir: String, + /// Metrics + pub metrics: ExecutionPlanMetricsSet, +} + +impl ShuffleWriterExec { + pub fn new( + stage_id: usize, + plan: Arc, + partitioning: Partitioning, + shuffle_dir: &str, + ) -> Self { + let partitioning = match partitioning { + Partitioning::Hash(expr, n) if expr.is_empty() => Partitioning::UnknownPartitioning(n), + Partitioning::Hash(expr, n) => { + // workaround for DataFusion bug https://github.com/apache/arrow-datafusion/issues/5184 + Partitioning::Hash( + expr.into_iter() + .filter(|e| e.as_any().downcast_ref::().is_none()) + .collect(), + n, + ) + } + _ => partitioning, + }; + + Self { + stage_id, + plan, + partitioning, + shuffle_dir: shuffle_dir.to_string(), + metrics: ExecutionPlanMetricsSet::new(), + } + } +} + +impl ExecutionPlan for ShuffleWriterExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.plan.schema() + } + + fn output_partitioning(&self) -> Partitioning { + self.partitioning.clone() + } + + fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + // TODO in the case of a single partition of a sorted plan this could be implemented + None + } + + fn children(&self) -> Vec> { + vec![self.plan.clone()] + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + unimplemented!() + } + + fn execute( + &self, + input_partition: usize, + context: Arc, + ) -> Result { + debug!( + "ShuffleWriterExec[stage={}].execute(input_partition={input_partition})", + self.stage_id + ); + + let mut stream = self.plan.execute(input_partition, context)?; + let write_time = + MetricBuilder::new(&self.metrics).subset_time("write_time", input_partition); + let repart_time = + MetricBuilder::new(&self.metrics).subset_time("repart_time", input_partition); + + let stage_id = self.stage_id; + let partitioning = self.output_partitioning(); + let partition_count = partitioning.partition_count(); + let shuffle_dir = self.shuffle_dir.clone(); + + let results = async move { + match &partitioning { + Partitioning::RoundRobinBatch(_) => { + unimplemented!() + } + Partitioning::UnknownPartitioning(_) => { + // stream the results from the query, preserving the input partitioning + let file = + format!("/{shuffle_dir}/shuffle_{stage_id}_{input_partition}_0.arrow"); + debug!("Executing query and writing results to {file}"); + let stats = write_stream_to_disk(&mut stream, &file, &write_time).await?; + debug!( + "Query completed. Shuffle write time: {}. Rows: {}.", + write_time, stats.num_rows + ); + } + Partitioning::Hash(_, _) => { + // we won't necessary produce output for every possible partition, so we + // create writers on demand + let mut writers: Vec> = vec![]; + for _ in 0..partition_count { + writers.push(None); + } + + let mut partitioner = + BatchPartitioner::try_new(partitioning, repart_time.clone())?; + + let mut rows = 0; + + while let Some(result) = stream.next().await { + let input_batch = result?; + rows += input_batch.num_rows(); + + debug!( + "ShuffleWriterExec[stage={}] writing batch:\n{}", + stage_id, + pretty_format_batches(&[input_batch.clone()])? + ); + + //write_metrics.input_rows.add(input_batch.num_rows()); + + partitioner.partition(input_batch, |output_partition, output_batch| { + match &mut writers[output_partition] { + Some(w) => { + w.write(&output_batch)?; + } + None => { + let path = format!( + "/{shuffle_dir}/shuffle_{stage_id}_{input_partition}_{output_partition}.arrow", + ); + let path = Path::new(&path); + debug!("ShuffleWriterExec[stage={}] Writing results to {:?}", stage_id, path); + + let mut writer = IPCWriter::new(path, stream.schema().as_ref())?; + + writer.write(&output_batch)?; + writers[output_partition] = Some(writer); + } + } + Ok(()) + })?; + } + + for (i, w) in writers.iter_mut().enumerate() { + match w { + Some(w) => { + w.finish()?; + debug!( + "ShuffleWriterExec[stage={}] Finished writing shuffle partition {} at {:?}. Batches: {}. Rows: {}. Bytes: {}.", + stage_id, + i, + w.path(), + w.num_batches, + w.num_rows, + w.num_bytes + ); + } + None => {} + } + } + debug!( + "ShuffleWriterExec[stage={}] Finished processing stream with {rows} rows", + stage_id + ); + } + } + + // create a dummy batch to return - later this could be metadata about the + // shuffle partitions that were written out + let schema = Arc::new(Schema::new(vec![ + Field::new("shuffle_repart_time", DataType::Int32, true), + Field::new("shuffle_write_time", DataType::Int32, true), + ])); + let arr_repart_time = Int32Array::from(vec![repart_time.value() as i32]); + let arr_write_time = Int32Array::from(vec![write_time.value() as i32]); + let batch = RecordBatch::try_new( + schema.clone(), + vec![Arc::new(arr_repart_time), Arc::new(arr_write_time)], + )?; + + // return as a stream + MemoryStream::try_new(vec![batch], schema, None) + }; + let schema = self.schema(); + Ok(Box::pin(RecordBatchStreamAdapter::new( + schema, + futures::stream::once(results).try_flatten(), + ))) + } + + fn statistics(&self) -> Result { + Ok(Statistics::new_unknown(&self.schema())) + } +} + +impl DisplayAs for ShuffleWriterExec { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + write!( + f, + "ShuffleWriterExec(stage_id={}, output_partitioning={:?})", + self.stage_id, self.partitioning + ) + } +} + +/// Stream data to disk in Arrow IPC format +pub async fn write_stream_to_disk( + stream: &mut Pin>, + path: &str, + disk_write_metric: &metrics::Time, +) -> Result { + let file = File::create(path).unwrap(); + + /*.map_err(|e| { + error!("Failed to create partition file at {}: {:?}", path, e); + BallistaError::IoError(e) + })?;*/ + + let mut num_rows = 0; + let mut num_batches = 0; + let mut num_bytes = 0; + let mut writer = FileWriter::try_new(file, stream.schema().as_ref())?; + + while let Some(result) = stream.next().await { + let batch = result?; + + let batch_size_bytes: usize = batch_byte_size(&batch); + num_batches += 1; + num_rows += batch.num_rows(); + num_bytes += batch_size_bytes; + + let timer = disk_write_metric.timer(); + writer.write(&batch)?; + timer.done(); + } + let timer = disk_write_metric.timer(); + writer.finish()?; + timer.done(); + Ok(PartitionStats { + num_rows: num_rows as i64, + num_batches: num_batches as i64, + num_bytes: num_bytes as i64, + column_stats: vec![], + }) +} diff --git a/src/utils.rs b/src/utils.rs new file mode 100644 index 0000000..5e52d45 --- /dev/null +++ b/src/utils.rs @@ -0,0 +1,30 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use pyo3::Python; +use std::future::Future; +use tokio::runtime::Runtime; + +/// Utility to collect rust futures with GIL released +pub(crate) fn wait_for_future(py: Python, f: F) -> F::Output +where + F: Send, + F::Output: Send, +{ + let rt = Runtime::new().unwrap(); + py.allow_threads(|| rt.block_on(f)) +} diff --git a/testdata/expected-plans/q1.txt b/testdata/expected-plans/q1.txt new file mode 100644 index 0000000..2396d76 --- /dev/null +++ b/testdata/expected-plans/q1.txt @@ -0,0 +1,48 @@ +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")] + +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))] + 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] + +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))] + 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)) + diff --git a/testdata/expected-plans/q10.txt b/testdata/expected-plans/q10.txt new file mode 100644 index 0000000..039c96f --- /dev/null +++ b/testdata/expected-plans/q10.txt @@ -0,0 +1,134 @@ +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, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name + Inner Join: customer.c_nationkey = nation.n_nationkey + Projection: customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount + Inner Join: orders.o_orderkey = lineitem.l_orderkey + Projection: customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey + Inner Join: customer.c_custkey = orders.o_custkey + TableScan: customer projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment] + Projection: orders.o_orderkey, orders.o_custkey + Filter: orders.o_orderdate >= Date32("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")] + 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")] + TableScan: nation projection=[n_nationkey, n_name] + +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] + 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] + +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)) + 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)) + diff --git a/testdata/expected-plans/q11.txt b/testdata/expected-plans/q11.txt new file mode 100644 index 0000000..98c619a --- /dev/null +++ b/testdata/expected-plans/q11.txt @@ -0,0 +1,188 @@ +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")] + +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)] + 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] + +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 })] + 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)) + diff --git a/testdata/expected-plans/q12.txt b/testdata/expected-plans/q12.txt new file mode 100644 index 0000000..2ba8687 --- /dev/null +++ b/testdata/expected-plans/q12.txt @@ -0,0 +1,73 @@ +DataFusion Logical Plan +======================= + +Sort: lineitem.l_shipmode ASC NULLS LAST + Projection: lineitem.l_shipmode, SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END) AS high_line_count, SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END) AS low_line_count + Aggregate: groupBy=[[lineitem.l_shipmode]], aggr=[[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)]] + Projection: orders.o_orderpriority, lineitem.l_shipmode + Inner Join: orders.o_orderkey = lineitem.l_orderkey + TableScan: orders projection=[o_orderkey, o_orderpriority] + Projection: lineitem.l_orderkey, lineitem.l_shipmode + Filter: (lineitem.l_shipmode = Utf8("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")] + +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)] + 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] + +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)] + 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)) + diff --git a/testdata/expected-plans/q13.txt b/testdata/expected-plans/q13.txt new file mode 100644 index 0000000..9d2d3ab --- /dev/null +++ b/testdata/expected-plans/q13.txt @@ -0,0 +1,85 @@ +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%")] + +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] + +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)) + diff --git a/testdata/expected-plans/q14.txt b/testdata/expected-plans/q14.txt new file mode 100644 index 0000000..54aaed3 --- /dev/null +++ b/testdata/expected-plans/q14.txt @@ -0,0 +1,63 @@ +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 + 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")] + 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] + +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] + 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)) + diff --git a/testdata/expected-plans/q16.txt b/testdata/expected-plans/q16.txt new file mode 100644 index 0000000..d40a22c --- /dev/null +++ b/testdata/expected-plans/q16.txt @@ -0,0 +1,113 @@ +DataFusion Logical Plan +======================= + +Sort: supplier_cnt DESC NULLS FIRST, part.p_brand ASC NULLS LAST, part.p_type ASC NULLS LAST, part.p_size ASC NULLS LAST + Projection: 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=[[]] + LeftAnti Join: partsupp.ps_suppkey = __correlated_sq_1.s_suppkey + Projection: partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size + Inner Join: partsupp.ps_partkey = part.p_partkey + TableScan: partsupp projection=[ps_partkey, ps_suppkey] + Filter: part.p_brand != Utf8("Brand#14") AND part.p_type NOT LIKE Utf8("SMALL PLATED%") AND part.p_size IN ([Int32(14), Int32(6), Int32(5), Int32(31), Int32(49), Int32(15), Int32(41), Int32(47)]) + TableScan: part projection=[p_partkey, p_brand, p_type, p_size], partial_filters=[part.p_brand != Utf8("Brand#14"), part.p_type NOT LIKE Utf8("SMALL PLATED%"), part.p_size IN ([Int32(14), Int32(6), Int32(5), Int32(31), Int32(49), Int32(15), Int32(41), Int32(47)])] + SubqueryAlias: __correlated_sq_1 + Projection: supplier.s_suppkey AS 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)] + 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] + +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 })] + 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)) + diff --git a/testdata/expected-plans/q17.txt b/testdata/expected-plans/q17.txt new file mode 100644 index 0000000..3604601 --- /dev/null +++ b/testdata/expected-plans/q17.txt @@ -0,0 +1,100 @@ +DataFusion Logical Plan +======================= + +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] + +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] + 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 })] + 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] + +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] + 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 })] + 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)) + diff --git a/testdata/expected-plans/q18.txt b/testdata/expected-plans/q18.txt new file mode 100644 index 0000000..f2ae3d8 --- /dev/null +++ b/testdata/expected-plans/q18.txt @@ -0,0 +1,119 @@ +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)]] + 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 + Projection: customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate + Inner Join: customer.c_custkey = orders.o_custkey + TableScan: customer projection=[c_custkey, c_name] + TableScan: orders projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate] + TableScan: lineitem projection=[l_orderkey, l_quantity] + SubqueryAlias: __correlated_sq_1 + Projection: lineitem.l_orderkey AS l_orderkey + Filter: SUM(lineitem.l_quantity) > Decimal128(Some(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)] + 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] + +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] + 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)) + 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)) + diff --git a/testdata/expected-plans/q19.txt b/testdata/expected-plans/q19.txt new file mode 100644 index 0000000..e61791f --- /dev/null +++ b/testdata/expected-plans/q19.txt @@ -0,0 +1,76 @@ +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: 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)] + +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] + 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] + +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] + 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)) + diff --git a/testdata/expected-plans/q2.txt b/testdata/expected-plans/q2.txt new file mode 100644 index 0000000..64df0f1 --- /dev/null +++ b/testdata/expected-plans/q2.txt @@ -0,0 +1,274 @@ +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: nation.n_regionkey = region.r_regionkey + Projection: part.p_partkey, part.p_mfgr, partsupp.ps_supplycost, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey + Inner Join: supplier.s_nationkey = nation.n_nationkey + Projection: part.p_partkey, part.p_mfgr, partsupp.ps_supplycost, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment + Inner Join: partsupp.ps_suppkey = supplier.s_suppkey + Projection: part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, partsupp.ps_supplycost + Inner Join: part.p_partkey = partsupp.ps_partkey + Projection: part.p_partkey, part.p_mfgr + Filter: part.p_size = Int32(48) AND part.p_type LIKE Utf8("%TIN") + TableScan: part projection=[p_partkey, p_mfgr, p_type, p_size], partial_filters=[part.p_size = Int32(48), part.p_type LIKE Utf8("%TIN")] + TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] + TableScan: supplier projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] + TableScan: nation projection=[n_nationkey, n_name, n_regionkey] + Projection: region.r_regionkey + Filter: region.r_name = Utf8("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: partsupp.ps_partkey, partsupp.ps_supplycost + Inner Join: nation.n_regionkey = region.r_regionkey + Projection: partsupp.ps_partkey, partsupp.ps_supplycost, nation.n_regionkey + Inner Join: supplier.s_nationkey = nation.n_nationkey + Projection: partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey + Inner Join: partsupp.ps_suppkey = supplier.s_suppkey + TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] + TableScan: supplier projection=[s_suppkey, s_nationkey] + TableScan: nation projection=[n_nationkey, n_regionkey] + Projection: region.r_regionkey + Filter: region.r_name = Utf8("ASIA") + TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("ASIA")] + +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] + 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] + 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] + 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] + +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] + 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)) + 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)) + diff --git a/testdata/expected-plans/q20.txt b/testdata/expected-plans/q20.txt new file mode 100644 index 0000000..3cdbf1c --- /dev/null +++ b/testdata/expected-plans/q20.txt @@ -0,0 +1,166 @@ +DataFusion Logical Plan +======================= + +Sort: supplier.s_name ASC NULLS LAST + Projection: supplier.s_name, supplier.s_address + LeftSemi Join: supplier.s_suppkey = __correlated_sq_1.ps_suppkey + Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address + Inner Join: supplier.s_nationkey = nation.n_nationkey + TableScan: supplier projection=[s_suppkey, s_name, s_address, s_nationkey] + Projection: nation.n_nationkey + Filter: nation.n_name = Utf8("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")] + +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] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=LeftSemi, 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 + 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 + 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] + 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] + +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)) + 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)) + 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] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "ps_suppkey", index: 0 })] + 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)) + diff --git a/testdata/expected-plans/q21.txt b/testdata/expected-plans/q21.txt new file mode 100644 index 0000000..557f95f --- /dev/null +++ b/testdata/expected-plans/q21.txt @@ -0,0 +1,194 @@ +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 + LeftAnti Join: l1.l_orderkey = l3.l_orderkey Filter: l3.l_suppkey != l1.l_suppkey + LeftSemi Join: l1.l_orderkey = l2.l_orderkey Filter: l2.l_suppkey != l1.l_suppkey + Projection: supplier.s_name, l1.l_orderkey, l1.l_suppkey + Inner Join: supplier.s_nationkey = nation.n_nationkey + Projection: supplier.s_name, supplier.s_nationkey, l1.l_orderkey, l1.l_suppkey + Inner Join: l1.l_orderkey = orders.o_orderkey + Projection: supplier.s_name, supplier.s_nationkey, l1.l_orderkey, l1.l_suppkey + Inner Join: supplier.s_suppkey = l1.l_suppkey + TableScan: supplier projection=[s_suppkey, s_name, s_nationkey] + SubqueryAlias: l1 + Projection: lineitem.l_orderkey, lineitem.l_suppkey + Filter: lineitem.l_receiptdate > lineitem.l_commitdate + TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] + Projection: orders.o_orderkey + Filter: orders.o_orderstatus = Utf8("F") + TableScan: orders projection=[o_orderkey, o_orderstatus], partial_filters=[orders.o_orderstatus = Utf8("F")] + 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] + +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))] + 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] + 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] + 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_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] + +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] + 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 + ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "l_orderkey", index: 1 }], 4)) + 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)) + diff --git a/testdata/expected-plans/q22.txt b/testdata/expected-plans/q22.txt new file mode 100644 index 0000000..cdc613d --- /dev/null +++ b/testdata/expected-plans/q22.txt @@ -0,0 +1,114 @@ +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)]] + 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")])] + 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)] + +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] + +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)) + diff --git a/testdata/expected-plans/q3.txt b/testdata/expected-plans/q3.txt new file mode 100644 index 0000000..e6f9749 --- /dev/null +++ b/testdata/expected-plans/q3.txt @@ -0,0 +1,114 @@ +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: orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount + Inner Join: orders.o_orderkey = lineitem.l_orderkey + Projection: orders.o_orderkey, orders.o_orderdate, orders.o_shippriority + Inner Join: customer.c_custkey = orders.o_custkey + Projection: customer.c_custkey + Filter: customer.c_mktsegment = Utf8("BUILDING") + TableScan: customer projection=[c_custkey, c_mktsegment], 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")] + 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")] + +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)] + 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] + +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] + 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)) + 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)) + diff --git a/testdata/expected-plans/q4.txt b/testdata/expected-plans/q4.txt new file mode 100644 index 0000000..17a19e1 --- /dev/null +++ b/testdata/expected-plans/q4.txt @@ -0,0 +1,81 @@ +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 + LeftSemi Join: orders.o_orderkey = lineitem.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] + +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] + +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] + 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)) + diff --git a/testdata/expected-plans/q5.txt b/testdata/expected-plans/q5.txt new file mode 100644 index 0000000..ca7997e --- /dev/null +++ b/testdata/expected-plans/q5.txt @@ -0,0 +1,182 @@ +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: lineitem.l_extendedprice, lineitem.l_discount, nation.n_name + Inner Join: nation.n_regionkey = region.r_regionkey + Projection: lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_regionkey + Inner Join: supplier.s_nationkey = nation.n_nationkey + Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey + Inner Join: lineitem.l_suppkey = supplier.s_suppkey, customer.c_nationkey = supplier.s_nationkey + Projection: customer.c_nationkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount + Inner Join: orders.o_orderkey = lineitem.l_orderkey + Projection: customer.c_nationkey, orders.o_orderkey + Inner Join: customer.c_custkey = orders.o_custkey + TableScan: customer projection=[c_custkey, c_nationkey] + Projection: orders.o_orderkey, orders.o_custkey + Filter: orders.o_orderdate >= Date32("8766") AND orders.o_orderdate < Date32("9131") + TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("8766"), orders.o_orderdate < Date32("9131")] + TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount] + TableScan: supplier projection=[s_suppkey, s_nationkey] + TableScan: nation projection=[n_nationkey, n_name, n_regionkey] + Projection: region.r_regionkey + Filter: region.r_name = Utf8("AFRICA") + TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("AFRICA")] + +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] + 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] + +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] + 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)) + diff --git a/testdata/expected-plans/q6.txt b/testdata/expected-plans/q6.txt new file mode 100644 index 0000000..0f41810 --- /dev/null +++ b/testdata/expected-plans/q6.txt @@ -0,0 +1,38 @@ +DataFusion Logical Plan +======================= + +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)] + +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] + +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)] + 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)) + diff --git a/testdata/expected-plans/q7.txt b/testdata/expected-plans/q7.txt new file mode 100644 index 0000000..5b778e4 --- /dev/null +++ b/testdata/expected-plans/q7.txt @@ -0,0 +1,193 @@ +DataFusion Logical Plan +======================= + +Sort: shipping.supp_nation ASC NULLS LAST, shipping.cust_nation ASC NULLS LAST, shipping.l_year ASC NULLS LAST + Projection: shipping.supp_nation, shipping.cust_nation, shipping.l_year, SUM(shipping.volume) AS revenue + Aggregate: groupBy=[[shipping.supp_nation, shipping.cust_nation, shipping.l_year]], aggr=[[SUM(shipping.volume)]] + SubqueryAlias: shipping + Projection: n1.n_name AS supp_nation, n2.n_name AS cust_nation, 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")] + +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)] + 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 + 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] + +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 })] + 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] + 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)) + diff --git a/testdata/expected-plans/q8.txt b/testdata/expected-plans/q8.txt new file mode 100644 index 0000000..c315805 --- /dev/null +++ b/testdata/expected-plans/q8.txt @@ -0,0 +1,240 @@ +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)]] + 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 + 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 + Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_orderdate, n1.n_regionkey + Inner Join: customer.c_nationkey = n1.n_nationkey + Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_orderdate, customer.c_nationkey + Inner Join: orders.o_custkey = customer.c_custkey + Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_custkey, orders.o_orderdate + Inner Join: lineitem.l_orderkey = orders.o_orderkey + Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey + Inner Join: lineitem.l_suppkey = supplier.s_suppkey + Projection: lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount + Inner Join: part.p_partkey = lineitem.l_partkey + Projection: part.p_partkey + Filter: part.p_type = Utf8("LARGE PLATED STEEL") + 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")] + TableScan: customer projection=[c_custkey, c_nationkey] + SubqueryAlias: n1 + TableScan: nation projection=[n_nationkey, n_regionkey] + SubqueryAlias: n2 + TableScan: nation projection=[n_nationkey, n_name] + Projection: region.r_regionkey + Filter: region.r_name = Utf8("MIDDLE EAST") + TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("MIDDLE EAST")] + +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)] + 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 })] + 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] + 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] + 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] + +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)] + 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)) + diff --git a/testdata/expected-plans/q9.txt b/testdata/expected-plans/q9.txt new file mode 100644 index 0000000..5afd3dc --- /dev/null +++ b/testdata/expected-plans/q9.txt @@ -0,0 +1,174 @@ +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)]] + 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 + 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 + Projection: lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, partsupp.ps_supplycost + Inner Join: lineitem.l_suppkey = partsupp.ps_suppkey, lineitem.l_partkey = partsupp.ps_partkey + Projection: lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey + Inner Join: lineitem.l_suppkey = supplier.s_suppkey + Projection: lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount + Inner Join: part.p_partkey = lineitem.l_partkey + Projection: part.p_partkey + Filter: part.p_name LIKE Utf8("%moccasin%") + TableScan: part projection=[p_partkey, p_name], partial_filters=[part.p_name LIKE Utf8("%moccasin%")] + TableScan: lineitem projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount] + TableScan: supplier projection=[s_suppkey, s_nationkey] + TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] + TableScan: orders projection=[o_orderkey, o_orderdate] + TableScan: nation projection=[n_nationkey, n_name] + +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)] + 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 })] + 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] + 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_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 + 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] + +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)) + 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)) + diff --git a/testdata/queries/q1.sql b/testdata/queries/q1.sql new file mode 100644 index 0000000..0dc4c3e --- /dev/null +++ b/testdata/queries/q1.sql @@ -0,0 +1,23 @@ +-- SQLBench-H query 1 derived from TPC-H query 1 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + l_returnflag, + l_linestatus, + sum(l_quantity) as sum_qty, + sum(l_extendedprice) as sum_base_price, + sum(l_extendedprice * (1 - l_discount)) as sum_disc_price, + sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge, + avg(l_quantity) as avg_qty, + avg(l_extendedprice) as avg_price, + avg(l_discount) as avg_disc, + count(*) as count_order +from + lineitem +where + l_shipdate <= date '1998-12-01' - interval '68 days' +group by + l_returnflag, + l_linestatus +order by + l_returnflag, + l_linestatus; diff --git a/testdata/queries/q10.sql b/testdata/queries/q10.sql new file mode 100644 index 0000000..576338f --- /dev/null +++ b/testdata/queries/q10.sql @@ -0,0 +1,33 @@ +-- SQLBench-H query 10 derived from TPC-H query 10 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + c_custkey, + c_name, + sum(l_extendedprice * (1 - l_discount)) as revenue, + c_acctbal, + n_name, + c_address, + c_phone, + c_comment +from + customer, + orders, + lineitem, + nation +where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate >= date '1993-07-01' + and o_orderdate < date '1993-07-01' + interval '3' month + and l_returnflag = 'R' + and c_nationkey = n_nationkey +group by + c_custkey, + c_name, + c_acctbal, + c_phone, + n_name, + c_address, + c_comment +order by + revenue desc limit 20; diff --git a/testdata/queries/q11.sql b/testdata/queries/q11.sql new file mode 100644 index 0000000..f4ead45 --- /dev/null +++ b/testdata/queries/q11.sql @@ -0,0 +1,29 @@ +-- SQLBench-H query 11 derived from TPC-H query 11 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + ps_partkey, + sum(ps_supplycost * ps_availqty) as value +from + partsupp, + supplier, + nation +where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'ALGERIA' +group by + ps_partkey having + sum(ps_supplycost * ps_availqty) > ( + select + sum(ps_supplycost * ps_availqty) * 0.0001000000 + from + partsupp, + supplier, + nation + where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'ALGERIA' + ) +order by + value desc; diff --git a/testdata/queries/q12.sql b/testdata/queries/q12.sql new file mode 100644 index 0000000..4ab4ea6 --- /dev/null +++ b/testdata/queries/q12.sql @@ -0,0 +1,30 @@ +-- SQLBench-H query 12 derived from TPC-H query 12 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + l_shipmode, + sum(case + when o_orderpriority = '1-URGENT' + or o_orderpriority = '2-HIGH' + then 1 + else 0 + end) as high_line_count, + sum(case + when o_orderpriority <> '1-URGENT' + and o_orderpriority <> '2-HIGH' + then 1 + else 0 + end) as low_line_count +from + orders, + lineitem +where + o_orderkey = l_orderkey + and l_shipmode in ('FOB', 'SHIP') + and l_commitdate < l_receiptdate + and l_shipdate < l_commitdate + and l_receiptdate >= date '1995-01-01' + and l_receiptdate < date '1995-01-01' + interval '1' year +group by + l_shipmode +order by + l_shipmode; diff --git a/testdata/queries/q13.sql b/testdata/queries/q13.sql new file mode 100644 index 0000000..301e35d --- /dev/null +++ b/testdata/queries/q13.sql @@ -0,0 +1,22 @@ +-- SQLBench-H query 13 derived from TPC-H query 13 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + c_count, + count(*) as custdist +from + ( + select + c_custkey, + count(o_orderkey) + from + customer left outer join orders on + c_custkey = o_custkey + and o_comment not like '%express%requests%' + group by + c_custkey + ) as c_orders (c_custkey, c_count) +group by + c_count +order by + custdist desc, + c_count desc; diff --git a/testdata/queries/q14.sql b/testdata/queries/q14.sql new file mode 100644 index 0000000..6040ac7 --- /dev/null +++ b/testdata/queries/q14.sql @@ -0,0 +1,15 @@ +-- SQLBench-H query 14 derived from TPC-H query 14 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + 100.00 * sum(case + when p_type like 'PROMO%' + then l_extendedprice * (1 - l_discount) + else 0 + end) / sum(l_extendedprice * (1 - l_discount)) as promo_revenue +from + lineitem, + part +where + l_partkey = p_partkey + and l_shipdate >= date '1995-02-01' + and l_shipdate < date '1995-02-01' + interval '1' month; diff --git a/testdata/queries/q15.sql b/testdata/queries/q15.sql new file mode 100644 index 0000000..0fe03a7 --- /dev/null +++ b/testdata/queries/q15.sql @@ -0,0 +1,33 @@ +-- SQLBench-H query 15 derived from TPC-H query 15 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +create view revenue0 (supplier_no, total_revenue) as + select + l_suppkey, + sum(l_extendedprice * (1 - l_discount)) + from + lineitem + where + l_shipdate >= date '1996-08-01' + and l_shipdate < date '1996-08-01' + interval '3' month + group by + l_suppkey; +select + s_suppkey, + s_name, + s_address, + s_phone, + total_revenue +from + supplier, + revenue0 +where + s_suppkey = supplier_no + and total_revenue = ( + select + max(total_revenue) + from + revenue0 + ) +order by + s_suppkey; +drop view revenue0; diff --git a/testdata/queries/q16.sql b/testdata/queries/q16.sql new file mode 100644 index 0000000..7fdf365 --- /dev/null +++ b/testdata/queries/q16.sql @@ -0,0 +1,32 @@ +-- SQLBench-H query 16 derived from TPC-H query 16 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + p_brand, + p_type, + p_size, + count(distinct ps_suppkey) as supplier_cnt +from + partsupp, + part +where + p_partkey = ps_partkey + and p_brand <> 'Brand#14' + and p_type not like 'SMALL PLATED%' + and p_size in (14, 6, 5, 31, 49, 15, 41, 47) + and ps_suppkey not in ( + select + s_suppkey + from + supplier + where + s_comment like '%Customer%Complaints%' + ) +group by + p_brand, + p_type, + p_size +order by + supplier_cnt desc, + p_brand, + p_type, + p_size; diff --git a/testdata/queries/q17.sql b/testdata/queries/q17.sql new file mode 100644 index 0000000..ffa0f15 --- /dev/null +++ b/testdata/queries/q17.sql @@ -0,0 +1,19 @@ +-- SQLBench-H query 17 derived from TPC-H query 17 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + sum(l_extendedprice) / 7.0 as avg_yearly +from + lineitem, + part +where + p_partkey = l_partkey + and p_brand = 'Brand#42' + and p_container = 'LG BAG' + and l_quantity < ( + select + 0.2 * avg(l_quantity) + from + lineitem + where + l_partkey = p_partkey + ); diff --git a/testdata/queries/q18.sql b/testdata/queries/q18.sql new file mode 100644 index 0000000..f4ab194 --- /dev/null +++ b/testdata/queries/q18.sql @@ -0,0 +1,34 @@ +-- SQLBench-H query 18 derived from TPC-H query 18 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice, + sum(l_quantity) +from + customer, + orders, + lineitem +where + o_orderkey in ( + select + l_orderkey + from + lineitem + group by + l_orderkey having + sum(l_quantity) > 313 + ) + and c_custkey = o_custkey + and o_orderkey = l_orderkey +group by + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice +order by + o_totalprice desc, + o_orderdate limit 100; diff --git a/testdata/queries/q19.sql b/testdata/queries/q19.sql new file mode 100644 index 0000000..ad5fb7d --- /dev/null +++ b/testdata/queries/q19.sql @@ -0,0 +1,37 @@ +-- SQLBench-H query 19 derived from TPC-H query 19 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + sum(l_extendedprice* (1 - l_discount)) as revenue +from + lineitem, + part +where + ( + p_partkey = l_partkey + and p_brand = 'Brand#21' + and p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG') + and l_quantity >= 8 and l_quantity <= 8 + 10 + and p_size between 1 and 5 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#13' + and p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK') + and l_quantity >= 20 and l_quantity <= 20 + 10 + and p_size between 1 and 10 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#52' + and p_container in ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG') + and l_quantity >= 30 and l_quantity <= 30 + 10 + and p_size between 1 and 15 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ); diff --git a/testdata/queries/q2.sql b/testdata/queries/q2.sql new file mode 100644 index 0000000..2936532 --- /dev/null +++ b/testdata/queries/q2.sql @@ -0,0 +1,45 @@ +-- SQLBench-H query 2 derived from TPC-H query 2 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + s_acctbal, + s_name, + n_name, + p_partkey, + p_mfgr, + s_address, + s_phone, + s_comment +from + part, + supplier, + partsupp, + nation, + region +where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and p_size = 48 + and p_type like '%TIN' + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'ASIA' + and ps_supplycost = ( + select + min(ps_supplycost) + from + partsupp, + supplier, + nation, + region + where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'ASIA' + ) +order by + s_acctbal desc, + n_name, + s_name, + p_partkey limit 100; diff --git a/testdata/queries/q20.sql b/testdata/queries/q20.sql new file mode 100644 index 0000000..3136ca3 --- /dev/null +++ b/testdata/queries/q20.sql @@ -0,0 +1,39 @@ +-- SQLBench-H query 20 derived from TPC-H query 20 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + s_name, + s_address +from + supplier, + nation +where + s_suppkey in ( + select + ps_suppkey + from + partsupp + where + ps_partkey in ( + select + p_partkey + from + part + where + p_name like 'blanched%' + ) + and ps_availqty > ( + select + 0.5 * sum(l_quantity) + from + lineitem + where + l_partkey = ps_partkey + and l_suppkey = ps_suppkey + and l_shipdate >= date '1993-01-01' + and l_shipdate < date '1993-01-01' + interval '1' year + ) + ) + and s_nationkey = n_nationkey + and n_name = 'KENYA' +order by + s_name; diff --git a/testdata/queries/q21.sql b/testdata/queries/q21.sql new file mode 100644 index 0000000..0170469 --- /dev/null +++ b/testdata/queries/q21.sql @@ -0,0 +1,41 @@ +-- SQLBench-H query 21 derived from TPC-H query 21 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + s_name, + count(*) as numwait +from + supplier, + lineitem l1, + orders, + nation +where + s_suppkey = l1.l_suppkey + and o_orderkey = l1.l_orderkey + and o_orderstatus = 'F' + and l1.l_receiptdate > l1.l_commitdate + and exists ( + select + * + from + lineitem l2 + where + l2.l_orderkey = l1.l_orderkey + and l2.l_suppkey <> l1.l_suppkey + ) + and not exists ( + select + * + from + lineitem l3 + where + l3.l_orderkey = l1.l_orderkey + and l3.l_suppkey <> l1.l_suppkey + and l3.l_receiptdate > l3.l_commitdate + ) + and s_nationkey = n_nationkey + and n_name = 'ARGENTINA' +group by + s_name +order by + numwait desc, + s_name limit 100; diff --git a/testdata/queries/q22.sql b/testdata/queries/q22.sql new file mode 100644 index 0000000..8d528ef --- /dev/null +++ b/testdata/queries/q22.sql @@ -0,0 +1,39 @@ +-- SQLBench-H query 22 derived from TPC-H query 22 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + cntrycode, + count(*) as numcust, + sum(c_acctbal) as totacctbal +from + ( + select + substring(c_phone from 1 for 2) as cntrycode, + c_acctbal + from + customer + where + substring(c_phone from 1 for 2) in + ('24', '34', '16', '30', '33', '14', '13') + and c_acctbal > ( + select + avg(c_acctbal) + from + customer + where + c_acctbal > 0.00 + and substring(c_phone from 1 for 2) in + ('24', '34', '16', '30', '33', '14', '13') + ) + and not exists ( + select + * + from + orders + where + o_custkey = c_custkey + ) + ) as custsale +group by + cntrycode +order by + cntrycode; diff --git a/testdata/queries/q3.sql b/testdata/queries/q3.sql new file mode 100644 index 0000000..b60be7f --- /dev/null +++ b/testdata/queries/q3.sql @@ -0,0 +1,24 @@ +-- SQLBench-H query 3 derived from TPC-H query 3 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + l_orderkey, + sum(l_extendedprice * (1 - l_discount)) as revenue, + o_orderdate, + o_shippriority +from + customer, + orders, + lineitem +where + c_mktsegment = 'BUILDING' + and c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate < date '1995-03-15' + and l_shipdate > date '1995-03-15' +group by + l_orderkey, + o_orderdate, + o_shippriority +order by + revenue desc, + o_orderdate limit 10; diff --git a/testdata/queries/q4.sql b/testdata/queries/q4.sql new file mode 100644 index 0000000..05fae97 --- /dev/null +++ b/testdata/queries/q4.sql @@ -0,0 +1,23 @@ +-- SQLBench-H query 4 derived from TPC-H query 4 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + o_orderpriority, + count(*) as order_count +from + orders +where + o_orderdate >= date '1995-04-01' + and o_orderdate < date '1995-04-01' + interval '3' month + and exists ( + select + * + from + lineitem + where + l_orderkey = o_orderkey + and l_commitdate < l_receiptdate + ) +group by + o_orderpriority +order by + o_orderpriority; diff --git a/testdata/queries/q5.sql b/testdata/queries/q5.sql new file mode 100644 index 0000000..4b97ef0 --- /dev/null +++ b/testdata/queries/q5.sql @@ -0,0 +1,26 @@ +-- SQLBench-H query 5 derived from TPC-H query 5 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + n_name, + sum(l_extendedprice * (1 - l_discount)) as revenue +from + customer, + orders, + lineitem, + supplier, + nation, + region +where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and l_suppkey = s_suppkey + and c_nationkey = s_nationkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'AFRICA' + and o_orderdate >= date '1994-01-01' + and o_orderdate < date '1994-01-01' + interval '1' year +group by + n_name +order by + revenue desc; diff --git a/testdata/queries/q6.sql b/testdata/queries/q6.sql new file mode 100644 index 0000000..f5b4bae --- /dev/null +++ b/testdata/queries/q6.sql @@ -0,0 +1,11 @@ +-- SQLBench-H query 6 derived from TPC-H query 6 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + sum(l_extendedprice * l_discount) as revenue +from + lineitem +where + l_shipdate >= date '1994-01-01' + and l_shipdate < date '1994-01-01' + interval '1' year + and l_discount between 0.04 - 0.01 and 0.04 + 0.01 + and l_quantity < 24; diff --git a/testdata/queries/q7.sql b/testdata/queries/q7.sql new file mode 100644 index 0000000..f3919be --- /dev/null +++ b/testdata/queries/q7.sql @@ -0,0 +1,41 @@ +-- SQLBench-H query 7 derived from TPC-H query 7 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + supp_nation, + cust_nation, + l_year, + sum(volume) as revenue +from + ( + select + n1.n_name as supp_nation, + n2.n_name as cust_nation, + extract(year from l_shipdate) as l_year, + l_extendedprice * (1 - l_discount) as volume + from + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2 + where + s_suppkey = l_suppkey + and o_orderkey = l_orderkey + and c_custkey = o_custkey + and s_nationkey = n1.n_nationkey + and c_nationkey = n2.n_nationkey + and ( + (n1.n_name = 'GERMANY' and n2.n_name = 'IRAQ') + or (n1.n_name = 'IRAQ' and n2.n_name = 'GERMANY') + ) + and l_shipdate between date '1995-01-01' and date '1996-12-31' + ) as shipping +group by + supp_nation, + cust_nation, + l_year +order by + supp_nation, + cust_nation, + l_year; diff --git a/testdata/queries/q8.sql b/testdata/queries/q8.sql new file mode 100644 index 0000000..7c53e14 --- /dev/null +++ b/testdata/queries/q8.sql @@ -0,0 +1,39 @@ +-- SQLBench-H query 8 derived from TPC-H query 8 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + o_year, + sum(case + when nation = 'IRAQ' then volume + else 0 + end) / sum(volume) as mkt_share +from + ( + select + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) as volume, + n2.n_name as nation + from + part, + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2, + region + where + p_partkey = l_partkey + and s_suppkey = l_suppkey + and l_orderkey = o_orderkey + and o_custkey = c_custkey + and c_nationkey = n1.n_nationkey + and n1.n_regionkey = r_regionkey + and r_name = 'MIDDLE EAST' + and s_nationkey = n2.n_nationkey + and o_orderdate between date '1995-01-01' and date '1996-12-31' + and p_type = 'LARGE PLATED STEEL' + ) as all_nations +group by + o_year +order by + o_year; diff --git a/testdata/queries/q9.sql b/testdata/queries/q9.sql new file mode 100644 index 0000000..2455695 --- /dev/null +++ b/testdata/queries/q9.sql @@ -0,0 +1,34 @@ +-- SQLBench-H query 9 derived from TPC-H query 9 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + nation, + o_year, + sum(amount) as sum_profit +from + ( + select + n_name as nation, + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount + from + part, + supplier, + lineitem, + partsupp, + orders, + nation + where + s_suppkey = l_suppkey + and ps_suppkey = l_suppkey + and ps_partkey = l_partkey + and p_partkey = l_partkey + and o_orderkey = l_orderkey + and s_nationkey = n_nationkey + and p_name like '%moccasin%' + ) as profit +group by + nation, + o_year +order by + nation, + o_year desc;