├── .github └── workflows │ └── rust.yml ├── .gitignore ├── Cargo.lock ├── Cargo.toml ├── LICENSE ├── NOTICE ├── README.md ├── build.rs ├── docs ├── README.md ├── sqlbench-h-per-query.png └── sqlbench-h-total.png ├── examples ├── tips.csv ├── tips.parquet └── tips.py ├── pyproject.toml ├── raysql ├── __init__.py ├── context.py ├── main.py ├── ray_utils.py └── tests │ └── test_context.py ├── requirements-in.txt ├── src ├── context.rs ├── lib.rs ├── planner.rs ├── proto │ ├── datafusion.proto │ ├── generated │ │ ├── mod.rs │ │ └── protobuf.rs │ ├── mod.rs │ └── raysql.proto ├── query_stage.rs ├── shuffle │ ├── codec.rs │ ├── mod.rs │ ├── ray_shuffle │ │ ├── mod.rs │ │ ├── reader.rs │ │ └── writer.rs │ ├── reader.rs │ └── writer.rs └── utils.rs └── testdata ├── expected-plans ├── q1.txt ├── q10.txt ├── q11.txt ├── q12.txt ├── q13.txt ├── q14.txt ├── q16.txt ├── q17.txt ├── q18.txt ├── q19.txt ├── q2.txt ├── q20.txt ├── q21.txt ├── q22.txt ├── q3.txt ├── q4.txt ├── q5.txt ├── q6.txt ├── q7.txt ├── q8.txt └── q9.txt └── queries ├── q1.sql ├── q10.sql ├── q11.sql ├── q12.sql ├── q13.sql ├── q14.sql ├── q15.sql ├── q16.sql ├── q17.sql ├── q18.sql ├── q19.sql ├── q2.sql ├── q20.sql ├── q21.sql ├── q22.sql ├── q3.sql ├── q4.sql ├── q5.sql ├── q6.sql ├── q7.sql ├── q8.sql └── q9.sql /.github/workflows/rust.yml: -------------------------------------------------------------------------------- 1 | name: Rust 2 | 3 | on: 4 | push: 5 | branches: [ "main" ] 6 | pull_request: 7 | branches: [ "main" ] 8 | 9 | env: 10 | CARGO_TERM_COLOR: always 11 | 12 | jobs: 13 | build: 14 | 15 | runs-on: ubuntu-latest 16 | 17 | steps: 18 | - uses: actions/checkout@v3 19 | - name: Install protobuf compiler 20 | shell: bash 21 | run: sudo apt-get install protobuf-compiler 22 | - name: Build Rust code 23 | run: cargo build --verbose 24 | - name: Run tests 25 | run: cargo test --verbose 26 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | .idea 2 | target 3 | __pycache__ 4 | venv 5 | *.so 6 | *.log 7 | results-sf* 8 | -------------------------------------------------------------------------------- /Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "raysql" 3 | description = "RaySQL: DataFusion on Ray" 4 | homepage = "https://github.com/datafusion-contrib/ray-sql" 5 | repository = "https://github.com/datafusion-contrib/ray-sql" 6 | authors = ["Andy Grove ", "Frank Luan "] 7 | version = "0.6.0" 8 | edition = "2021" 9 | readme = "README.md" 10 | license = "Apache-2.0" 11 | rust-version = "1.62" 12 | build = "build.rs" 13 | 14 | [dependencies] 15 | datafusion = { version = "33.0.0", features = ["pyarrow", "avro"] } 16 | datafusion-proto = "33.0.0" 17 | datafusion-python = "33.0.0" 18 | futures = "0.3" 19 | glob = "0.3" 20 | log = "0.4" 21 | prost = "0.12" 22 | prost-types = "0.12" 23 | pyo3 = { version = "0.20", features = ["extension-module", "abi3", "abi3-py37"] } 24 | tokio = { version = "1.24", features = ["macros", "rt", "rt-multi-thread", "sync"] } 25 | uuid = "1.2" 26 | 27 | [build-dependencies] 28 | rustc_version = "0.4.0" 29 | tonic-build = { version = "0.8", default-features = false, features = ["transport", "prost"] } 30 | 31 | [lib] 32 | name = "raysql" 33 | crate-type = ["cdylib", "rlib"] 34 | 35 | [package.metadata.maturin] 36 | name = "raysql._raysql_internal" 37 | 38 | [profile.release] 39 | codegen-units = 1 40 | lto = true -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Apache License 2 | Version 2.0, January 2004 3 | http://www.apache.org/licenses/ 4 | 5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 6 | 7 | 1. Definitions. 8 | 9 | "License" shall mean the terms and conditions for use, reproduction, 10 | and distribution as defined by Sections 1 through 9 of this document. 11 | 12 | "Licensor" shall mean the copyright owner or entity authorized by 13 | the copyright owner that is granting the License. 14 | 15 | "Legal Entity" shall mean the union of the acting entity and all 16 | other entities that control, are controlled by, or are under common 17 | control with that entity. For the purposes of this definition, 18 | "control" means (i) the power, direct or indirect, to cause the 19 | direction or management of such entity, whether by contract or 20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 21 | outstanding shares, or (iii) beneficial ownership of such entity. 22 | 23 | "You" (or "Your") shall mean an individual or Legal Entity 24 | exercising permissions granted by this License. 25 | 26 | "Source" form shall mean the preferred form for making modifications, 27 | including but not limited to software source code, documentation 28 | source, and configuration files. 29 | 30 | "Object" form shall mean any form resulting from mechanical 31 | transformation or translation of a Source form, including but 32 | not limited to compiled object code, generated documentation, 33 | and conversions to other media types. 34 | 35 | "Work" shall mean the work of authorship, whether in Source or 36 | Object form, made available under the License, as indicated by a 37 | copyright notice that is included in or attached to the work 38 | (an example is provided in the Appendix below). 39 | 40 | "Derivative Works" shall mean any work, whether in Source or Object 41 | form, that is based on (or derived from) the Work and for which the 42 | editorial revisions, annotations, elaborations, or other modifications 43 | represent, as a whole, an original work of authorship. For the purposes 44 | of this License, Derivative Works shall not include works that remain 45 | separable from, or merely link (or bind by name) to the interfaces of, 46 | the Work and Derivative Works thereof. 47 | 48 | "Contribution" shall mean any work of authorship, including 49 | the original version of the Work and any modifications or additions 50 | to that Work or Derivative Works thereof, that is intentionally 51 | submitted to Licensor for inclusion in the Work by the copyright owner 52 | or by an individual or Legal Entity authorized to submit on behalf of 53 | the copyright owner. For the purposes of this definition, "submitted" 54 | means any form of electronic, verbal, or written communication sent 55 | to the Licensor or its representatives, including but not limited to 56 | communication on electronic mailing lists, source code control systems, 57 | and issue tracking systems that are managed by, or on behalf of, the 58 | Licensor for the purpose of discussing and improving the Work, but 59 | excluding communication that is conspicuously marked or otherwise 60 | designated in writing by the copyright owner as "Not a Contribution." 61 | 62 | "Contributor" shall mean Licensor and any individual or Legal Entity 63 | on behalf of whom a Contribution has been received by Licensor and 64 | subsequently incorporated within the Work. 65 | 66 | 2. Grant of Copyright License. Subject to the terms and conditions of 67 | this License, each Contributor hereby grants to You a perpetual, 68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 69 | copyright license to reproduce, prepare Derivative Works of, 70 | publicly display, publicly perform, sublicense, and distribute the 71 | Work and such Derivative Works in Source or Object form. 72 | 73 | 3. Grant of Patent License. Subject to the terms and conditions of 74 | this License, each Contributor hereby grants to You a perpetual, 75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 76 | (except as stated in this section) patent license to make, have made, 77 | use, offer to sell, sell, import, and otherwise transfer the Work, 78 | where such license applies only to those patent claims licensable 79 | by such Contributor that are necessarily infringed by their 80 | Contribution(s) alone or by combination of their Contribution(s) 81 | with the Work to which such Contribution(s) was submitted. If You 82 | institute patent litigation against any entity (including a 83 | cross-claim or counterclaim in a lawsuit) alleging that the Work 84 | or a Contribution incorporated within the Work constitutes direct 85 | or contributory patent infringement, then any patent licenses 86 | granted to You under this License for that Work shall terminate 87 | as of the date such litigation is filed. 88 | 89 | 4. Redistribution. You may reproduce and distribute copies of the 90 | Work or Derivative Works thereof in any medium, with or without 91 | modifications, and in Source or Object form, provided that You 92 | meet the following conditions: 93 | 94 | (a) You must give any other recipients of the Work or 95 | Derivative Works a copy of this License; and 96 | 97 | (b) You must cause any modified files to carry prominent notices 98 | stating that You changed the files; and 99 | 100 | (c) You must retain, in the Source form of any Derivative Works 101 | that You distribute, all copyright, patent, trademark, and 102 | attribution notices from the Source form of the Work, 103 | excluding those notices that do not pertain to any part of 104 | the Derivative Works; and 105 | 106 | (d) If the Work includes a "NOTICE" text file as part of its 107 | distribution, then any Derivative Works that You distribute must 108 | include a readable copy of the attribution notices contained 109 | within such NOTICE file, excluding those notices that do not 110 | pertain to any part of the Derivative Works, in at least one 111 | of the following places: within a NOTICE text file distributed 112 | as part of the Derivative Works; within the Source form or 113 | documentation, if provided along with the Derivative Works; or, 114 | within a display generated by the Derivative Works, if and 115 | wherever such third-party notices normally appear. The contents 116 | of the NOTICE file are for informational purposes only and 117 | do not modify the License. You may add Your own attribution 118 | notices within Derivative Works that You distribute, alongside 119 | or as an addendum to the NOTICE text from the Work, provided 120 | that such additional attribution notices cannot be construed 121 | as modifying the License. 122 | 123 | You may add Your own copyright statement to Your modifications and 124 | may provide additional or different license terms and conditions 125 | for use, reproduction, or distribution of Your modifications, or 126 | for any such Derivative Works as a whole, provided Your use, 127 | reproduction, and distribution of the Work otherwise complies with 128 | the conditions stated in this License. 129 | 130 | 5. Submission of Contributions. Unless You explicitly state otherwise, 131 | any Contribution intentionally submitted for inclusion in the Work 132 | by You to the Licensor shall be under the terms and conditions of 133 | this License, without any additional terms or conditions. 134 | Notwithstanding the above, nothing herein shall supersede or modify 135 | the terms of any separate license agreement you may have executed 136 | with Licensor regarding such Contributions. 137 | 138 | 6. Trademarks. This License does not grant permission to use the trade 139 | names, trademarks, service marks, or product names of the Licensor, 140 | except as required for reasonable and customary use in describing the 141 | origin of the Work and reproducing the content of the NOTICE file. 142 | 143 | 7. Disclaimer of Warranty. Unless required by applicable law or 144 | agreed to in writing, Licensor provides the Work (and each 145 | Contributor provides its Contributions) on an "AS IS" BASIS, 146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 147 | implied, including, without limitation, any warranties or conditions 148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 149 | PARTICULAR PURPOSE. You are solely responsible for determining the 150 | appropriateness of using or redistributing the Work and assume any 151 | risks associated with Your exercise of permissions under this License. 152 | 153 | 8. Limitation of Liability. In no event and under no legal theory, 154 | whether in tort (including negligence), contract, or otherwise, 155 | unless required by applicable law (such as deliberate and grossly 156 | negligent acts) or agreed to in writing, shall any Contributor be 157 | liable to You for damages, including any direct, indirect, special, 158 | incidental, or consequential damages of any character arising as a 159 | result of this License or out of the use or inability to use the 160 | Work (including but not limited to damages for loss of goodwill, 161 | work stoppage, computer failure or malfunction, or any and all 162 | other commercial damages or losses), even if such Contributor 163 | has been advised of the possibility of such damages. 164 | 165 | 9. Accepting Warranty or Additional Liability. While redistributing 166 | the Work or Derivative Works thereof, You may choose to offer, 167 | and charge a fee for, acceptance of support, warranty, indemnity, 168 | or other liability obligations and/or rights consistent with this 169 | License. However, in accepting such obligations, You may act only 170 | on Your own behalf and on Your sole responsibility, not on behalf 171 | of any other Contributor, and only if You agree to indemnify, 172 | defend, and hold each Contributor harmless for any liability 173 | incurred by, or claims asserted against, such Contributor by reason 174 | of your accepting any such warranty or additional liability. 175 | 176 | END OF TERMS AND CONDITIONS 177 | 178 | APPENDIX: How to apply the Apache License to your work. 179 | 180 | To apply the Apache License to your work, attach the following 181 | boilerplate notice, with the fields enclosed by brackets "[]" 182 | replaced with your own identifying information. (Don't include 183 | the brackets!) The text should be enclosed in the appropriate 184 | comment syntax for the file format. We also recommend that a 185 | file or class name and description of purpose be included on the 186 | same "printed page" as the copyright notice for easier 187 | identification within third-party archives. 188 | 189 | Copyright [yyyy] [name of copyright owner] 190 | 191 | Licensed under the Apache License, Version 2.0 (the "License"); 192 | you may not use this file except in compliance with the License. 193 | You may obtain a copy of the License at 194 | 195 | http://www.apache.org/licenses/LICENSE-2.0 196 | 197 | Unless required by applicable law or agreed to in writing, software 198 | distributed under the License is distributed on an "AS IS" BASIS, 199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 200 | See the License for the specific language governing permissions and 201 | limitations under the License. 202 | -------------------------------------------------------------------------------- /NOTICE: -------------------------------------------------------------------------------- 1 | This product includes software developed at 2 | The Apache Software Foundation (http://www.apache.org/). -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # RaySQL: DataFusion on Ray 2 | 3 | This was a research project to evaluate performing distributed SQL queries from Python, using 4 | [Ray](https://www.ray.io/) and [DataFusion](https://github.com/apache/arrow-datafusion). 5 | 6 | ** The code has now moved to https://github.com/apache/datafusion-ray ** 7 | 8 | ## Goals 9 | 10 | - Demonstrate how easily new systems can be built on top of DataFusion. See the [design documentation](./docs/README.md) 11 | to understand how RaySQL works. 12 | - Drive requirements for DataFusion's [Python bindings](https://github.com/apache/arrow-datafusion-python). 13 | - Create content for an interesting blog post or conference talk. 14 | 15 | ## Non Goals 16 | 17 | - Build and support a production system. 18 | 19 | ## Example 20 | 21 | Run the following example live in your browser using a Google Colab [notebook](https://colab.research.google.com/drive/1tmSX0Lu6UFh58_-DBUVoyYx6BoXHOszP?usp=sharing). 22 | 23 | ```python 24 | import os 25 | import pandas as pd 26 | import ray 27 | 28 | from raysql import RaySqlContext 29 | 30 | SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) 31 | 32 | # Start a local cluster 33 | ray.init(resources={"worker": 1}) 34 | 35 | # Create a context and register a table 36 | ctx = RaySqlContext(2, use_ray_shuffle=True) 37 | # Register either a CSV or Parquet file 38 | # ctx.register_csv("tips", f"{SCRIPT_DIR}/tips.csv", True) 39 | ctx.register_parquet("tips", f"{SCRIPT_DIR}/tips.parquet") 40 | 41 | result_set = ctx.sql( 42 | "select sex, smoker, avg(tip/total_bill) as tip_pct from tips group by sex, smoker" 43 | ) 44 | for record_batch in result_set: 45 | print(record_batch.to_pandas()) 46 | ``` 47 | 48 | ## Status 49 | 50 | - RaySQL can run all queries in the TPC-H benchmark 51 | 52 | ## Features 53 | 54 | - Mature SQL support (CTEs, joins, subqueries, etc) thanks to DataFusion 55 | - Support for CSV and Parquet files 56 | 57 | ## Limitations 58 | 59 | - Requires a shared file system currently 60 | 61 | ## Performance 62 | 63 | This chart shows the performance of RaySQL compared to Apache Spark for 64 | [SQLBench-H](https://sqlbenchmarks.io/sqlbench-h/) at a very small data set (10GB), running on a desktop (Threadripper 65 | with 24 physical cores). Both RaySQL and Spark are configured with 24 executors. 66 | 67 | ### Overall Time 68 | 69 | RaySQL is ~1.9x faster overall for this scale factor and environment with disk-based shuffle. 70 | 71 | ![SQLBench-H Total](./docs/sqlbench-h-total.png) 72 | 73 | ### Per Query Time 74 | 75 | Spark is much faster on some queries, likely due to broadcast exchanges, which RaySQL hasn't implemented yet. 76 | 77 | ![SQLBench-H Per Query](./docs/sqlbench-h-per-query.png) 78 | 79 | ### Performance Plan 80 | 81 | I'm planning on experimenting with the following changes to improve performance: 82 | 83 | - Make better use of Ray futures to run more tasks in parallel 84 | - Use Ray object store for shuffle data transfer to reduce disk I/O cost 85 | - Keep upgrading to newer versions of DataFusion to pick up the latest optimizations 86 | 87 | ## Building 88 | 89 | ```bash 90 | # prepare development environment (used to build wheel / install in development) 91 | python3 -m venv venv 92 | # activate the venv 93 | source venv/bin/activate 94 | # update pip itself if necessary 95 | python -m pip install -U pip 96 | # install dependencies (for Python 3.8+) 97 | python -m pip install -r requirements-in.txt 98 | ``` 99 | 100 | Whenever rust code changes (your changes or via `git pull`): 101 | 102 | ```bash 103 | # make sure you activate the venv using "source venv/bin/activate" first 104 | maturin develop 105 | python -m pytest 106 | ``` 107 | 108 | ## Benchmarking 109 | 110 | Create a release build when running benchmarks, then use pip to install the wheel. 111 | 112 | ```bash 113 | maturin develop --release 114 | ``` 115 | 116 | ## How to update dependencies 117 | 118 | To change test dependencies, change the `requirements.in` and run 119 | 120 | ```bash 121 | # install pip-tools (this can be done only once), also consider running in venv 122 | python -m pip install pip-tools 123 | python -m piptools compile --generate-hashes -o requirements-310.txt 124 | ``` 125 | 126 | To update dependencies, run with `-U` 127 | 128 | ```bash 129 | python -m piptools compile -U --generate-hashes -o requirements-310.txt 130 | ``` 131 | 132 | More details [here](https://github.com/jazzband/pip-tools) 133 | -------------------------------------------------------------------------------- /build.rs: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | use std::path::Path; 19 | 20 | fn main() -> Result<(), String> { 21 | use std::io::Write; 22 | 23 | let out = std::path::PathBuf::from(std::env::var("OUT_DIR").unwrap()); 24 | 25 | // for use in docker build where file changes can be wonky 26 | println!("cargo:rerun-if-env-changed=FORCE_REBUILD"); 27 | 28 | let version = rustc_version::version().unwrap(); 29 | println!("cargo:rustc-env=RUSTC_VERSION={version}"); 30 | 31 | let path = "src/proto/generated/protobuf.rs"; 32 | 33 | // We don't include the proto files in releases so that downstreams 34 | // do not need to have PROTOC included 35 | if Path::new("src/proto/raysql.proto").exists() { 36 | println!("cargo:rerun-if-changed=src/proto/datafusion.proto"); 37 | println!("cargo:rerun-if-changed=src/proto/raysql.proto"); 38 | tonic_build::configure() 39 | .extern_path(".datafusion", "::datafusion_proto::protobuf") 40 | .compile(&["src/proto/raysql.proto"], &["src/proto"]) 41 | .map_err(|e| format!("protobuf compilation failed: {e}"))?; 42 | let generated_source_path = out.join("raysql.protobuf.rs"); 43 | let code = std::fs::read_to_string(generated_source_path).unwrap(); 44 | let mut file = std::fs::OpenOptions::new() 45 | .write(true) 46 | .truncate(true) 47 | .create(true) 48 | .open(path) 49 | .unwrap(); 50 | file.write_all(code.as_str().as_ref()).unwrap(); 51 | } 52 | 53 | Ok(()) 54 | } 55 | -------------------------------------------------------------------------------- /docs/sqlbench-h-per-query.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/datafusion-contrib/ray-sql/39a1fef446ecc6c27c1a0ee6039453522d365114/docs/sqlbench-h-per-query.png -------------------------------------------------------------------------------- /docs/sqlbench-h-total.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/datafusion-contrib/ray-sql/39a1fef446ecc6c27c1a0ee6039453522d365114/docs/sqlbench-h-total.png -------------------------------------------------------------------------------- /examples/tips.csv: -------------------------------------------------------------------------------- 1 | total_bill,tip,sex,smoker,day,time,size 2 | 16.99,1.01,Female,No,Sun,Dinner,2 3 | 10.34,1.66,Male,No,Sun,Dinner,3 4 | 21.01,3.5,Male,No,Sun,Dinner,3 5 | 23.68,3.31,Male,No,Sun,Dinner,2 6 | 24.59,3.61,Female,No,Sun,Dinner,4 7 | 25.29,4.71,Male,No,Sun,Dinner,4 8 | 8.77,2.0,Male,No,Sun,Dinner,2 9 | 26.88,3.12,Male,No,Sun,Dinner,4 10 | 15.04,1.96,Male,No,Sun,Dinner,2 11 | 14.78,3.23,Male,No,Sun,Dinner,2 12 | 10.27,1.71,Male,No,Sun,Dinner,2 13 | 35.26,5.0,Female,No,Sun,Dinner,4 14 | 15.42,1.57,Male,No,Sun,Dinner,2 15 | 18.43,3.0,Male,No,Sun,Dinner,4 16 | 14.83,3.02,Female,No,Sun,Dinner,2 17 | 21.58,3.92,Male,No,Sun,Dinner,2 18 | 10.33,1.67,Female,No,Sun,Dinner,3 19 | 16.29,3.71,Male,No,Sun,Dinner,3 20 | 16.97,3.5,Female,No,Sun,Dinner,3 21 | 20.65,3.35,Male,No,Sat,Dinner,3 22 | 17.92,4.08,Male,No,Sat,Dinner,2 23 | 20.29,2.75,Female,No,Sat,Dinner,2 24 | 15.77,2.23,Female,No,Sat,Dinner,2 25 | 39.42,7.58,Male,No,Sat,Dinner,4 26 | 19.82,3.18,Male,No,Sat,Dinner,2 27 | 17.81,2.34,Male,No,Sat,Dinner,4 28 | 13.37,2.0,Male,No,Sat,Dinner,2 29 | 12.69,2.0,Male,No,Sat,Dinner,2 30 | 21.7,4.3,Male,No,Sat,Dinner,2 31 | 19.65,3.0,Female,No,Sat,Dinner,2 32 | 9.55,1.45,Male,No,Sat,Dinner,2 33 | 18.35,2.5,Male,No,Sat,Dinner,4 34 | 15.06,3.0,Female,No,Sat,Dinner,2 35 | 20.69,2.45,Female,No,Sat,Dinner,4 36 | 17.78,3.27,Male,No,Sat,Dinner,2 37 | 24.06,3.6,Male,No,Sat,Dinner,3 38 | 16.31,2.0,Male,No,Sat,Dinner,3 39 | 16.93,3.07,Female,No,Sat,Dinner,3 40 | 18.69,2.31,Male,No,Sat,Dinner,3 41 | 31.27,5.0,Male,No,Sat,Dinner,3 42 | 16.04,2.24,Male,No,Sat,Dinner,3 43 | 17.46,2.54,Male,No,Sun,Dinner,2 44 | 13.94,3.06,Male,No,Sun,Dinner,2 45 | 9.68,1.32,Male,No,Sun,Dinner,2 46 | 30.4,5.6,Male,No,Sun,Dinner,4 47 | 18.29,3.0,Male,No,Sun,Dinner,2 48 | 22.23,5.0,Male,No,Sun,Dinner,2 49 | 32.4,6.0,Male,No,Sun,Dinner,4 50 | 28.55,2.05,Male,No,Sun,Dinner,3 51 | 18.04,3.0,Male,No,Sun,Dinner,2 52 | 12.54,2.5,Male,No,Sun,Dinner,2 53 | 10.29,2.6,Female,No,Sun,Dinner,2 54 | 34.81,5.2,Female,No,Sun,Dinner,4 55 | 9.94,1.56,Male,No,Sun,Dinner,2 56 | 25.56,4.34,Male,No,Sun,Dinner,4 57 | 19.49,3.51,Male,No,Sun,Dinner,2 58 | 38.01,3.0,Male,Yes,Sat,Dinner,4 59 | 26.41,1.5,Female,No,Sat,Dinner,2 60 | 11.24,1.76,Male,Yes,Sat,Dinner,2 61 | 48.27,6.73,Male,No,Sat,Dinner,4 62 | 20.29,3.21,Male,Yes,Sat,Dinner,2 63 | 13.81,2.0,Male,Yes,Sat,Dinner,2 64 | 11.02,1.98,Male,Yes,Sat,Dinner,2 65 | 18.29,3.76,Male,Yes,Sat,Dinner,4 66 | 17.59,2.64,Male,No,Sat,Dinner,3 67 | 20.08,3.15,Male,No,Sat,Dinner,3 68 | 16.45,2.47,Female,No,Sat,Dinner,2 69 | 3.07,1.0,Female,Yes,Sat,Dinner,1 70 | 20.23,2.01,Male,No,Sat,Dinner,2 71 | 15.01,2.09,Male,Yes,Sat,Dinner,2 72 | 12.02,1.97,Male,No,Sat,Dinner,2 73 | 17.07,3.0,Female,No,Sat,Dinner,3 74 | 26.86,3.14,Female,Yes,Sat,Dinner,2 75 | 25.28,5.0,Female,Yes,Sat,Dinner,2 76 | 14.73,2.2,Female,No,Sat,Dinner,2 77 | 10.51,1.25,Male,No,Sat,Dinner,2 78 | 17.92,3.08,Male,Yes,Sat,Dinner,2 79 | 27.2,4.0,Male,No,Thur,Lunch,4 80 | 22.76,3.0,Male,No,Thur,Lunch,2 81 | 17.29,2.71,Male,No,Thur,Lunch,2 82 | 19.44,3.0,Male,Yes,Thur,Lunch,2 83 | 16.66,3.4,Male,No,Thur,Lunch,2 84 | 10.07,1.83,Female,No,Thur,Lunch,1 85 | 32.68,5.0,Male,Yes,Thur,Lunch,2 86 | 15.98,2.03,Male,No,Thur,Lunch,2 87 | 34.83,5.17,Female,No,Thur,Lunch,4 88 | 13.03,2.0,Male,No,Thur,Lunch,2 89 | 18.28,4.0,Male,No,Thur,Lunch,2 90 | 24.71,5.85,Male,No,Thur,Lunch,2 91 | 21.16,3.0,Male,No,Thur,Lunch,2 92 | 28.97,3.0,Male,Yes,Fri,Dinner,2 93 | 22.49,3.5,Male,No,Fri,Dinner,2 94 | 5.75,1.0,Female,Yes,Fri,Dinner,2 95 | 16.32,4.3,Female,Yes,Fri,Dinner,2 96 | 22.75,3.25,Female,No,Fri,Dinner,2 97 | 40.17,4.73,Male,Yes,Fri,Dinner,4 98 | 27.28,4.0,Male,Yes,Fri,Dinner,2 99 | 12.03,1.5,Male,Yes,Fri,Dinner,2 100 | 21.01,3.0,Male,Yes,Fri,Dinner,2 101 | 12.46,1.5,Male,No,Fri,Dinner,2 102 | 11.35,2.5,Female,Yes,Fri,Dinner,2 103 | 15.38,3.0,Female,Yes,Fri,Dinner,2 104 | 44.3,2.5,Female,Yes,Sat,Dinner,3 105 | 22.42,3.48,Female,Yes,Sat,Dinner,2 106 | 20.92,4.08,Female,No,Sat,Dinner,2 107 | 15.36,1.64,Male,Yes,Sat,Dinner,2 108 | 20.49,4.06,Male,Yes,Sat,Dinner,2 109 | 25.21,4.29,Male,Yes,Sat,Dinner,2 110 | 18.24,3.76,Male,No,Sat,Dinner,2 111 | 14.31,4.0,Female,Yes,Sat,Dinner,2 112 | 14.0,3.0,Male,No,Sat,Dinner,2 113 | 7.25,1.0,Female,No,Sat,Dinner,1 114 | 38.07,4.0,Male,No,Sun,Dinner,3 115 | 23.95,2.55,Male,No,Sun,Dinner,2 116 | 25.71,4.0,Female,No,Sun,Dinner,3 117 | 17.31,3.5,Female,No,Sun,Dinner,2 118 | 29.93,5.07,Male,No,Sun,Dinner,4 119 | 10.65,1.5,Female,No,Thur,Lunch,2 120 | 12.43,1.8,Female,No,Thur,Lunch,2 121 | 24.08,2.92,Female,No,Thur,Lunch,4 122 | 11.69,2.31,Male,No,Thur,Lunch,2 123 | 13.42,1.68,Female,No,Thur,Lunch,2 124 | 14.26,2.5,Male,No,Thur,Lunch,2 125 | 15.95,2.0,Male,No,Thur,Lunch,2 126 | 12.48,2.52,Female,No,Thur,Lunch,2 127 | 29.8,4.2,Female,No,Thur,Lunch,6 128 | 8.52,1.48,Male,No,Thur,Lunch,2 129 | 14.52,2.0,Female,No,Thur,Lunch,2 130 | 11.38,2.0,Female,No,Thur,Lunch,2 131 | 22.82,2.18,Male,No,Thur,Lunch,3 132 | 19.08,1.5,Male,No,Thur,Lunch,2 133 | 20.27,2.83,Female,No,Thur,Lunch,2 134 | 11.17,1.5,Female,No,Thur,Lunch,2 135 | 12.26,2.0,Female,No,Thur,Lunch,2 136 | 18.26,3.25,Female,No,Thur,Lunch,2 137 | 8.51,1.25,Female,No,Thur,Lunch,2 138 | 10.33,2.0,Female,No,Thur,Lunch,2 139 | 14.15,2.0,Female,No,Thur,Lunch,2 140 | 16.0,2.0,Male,Yes,Thur,Lunch,2 141 | 13.16,2.75,Female,No,Thur,Lunch,2 142 | 17.47,3.5,Female,No,Thur,Lunch,2 143 | 34.3,6.7,Male,No,Thur,Lunch,6 144 | 41.19,5.0,Male,No,Thur,Lunch,5 145 | 27.05,5.0,Female,No,Thur,Lunch,6 146 | 16.43,2.3,Female,No,Thur,Lunch,2 147 | 8.35,1.5,Female,No,Thur,Lunch,2 148 | 18.64,1.36,Female,No,Thur,Lunch,3 149 | 11.87,1.63,Female,No,Thur,Lunch,2 150 | 9.78,1.73,Male,No,Thur,Lunch,2 151 | 7.51,2.0,Male,No,Thur,Lunch,2 152 | 14.07,2.5,Male,No,Sun,Dinner,2 153 | 13.13,2.0,Male,No,Sun,Dinner,2 154 | 17.26,2.74,Male,No,Sun,Dinner,3 155 | 24.55,2.0,Male,No,Sun,Dinner,4 156 | 19.77,2.0,Male,No,Sun,Dinner,4 157 | 29.85,5.14,Female,No,Sun,Dinner,5 158 | 48.17,5.0,Male,No,Sun,Dinner,6 159 | 25.0,3.75,Female,No,Sun,Dinner,4 160 | 13.39,2.61,Female,No,Sun,Dinner,2 161 | 16.49,2.0,Male,No,Sun,Dinner,4 162 | 21.5,3.5,Male,No,Sun,Dinner,4 163 | 12.66,2.5,Male,No,Sun,Dinner,2 164 | 16.21,2.0,Female,No,Sun,Dinner,3 165 | 13.81,2.0,Male,No,Sun,Dinner,2 166 | 17.51,3.0,Female,Yes,Sun,Dinner,2 167 | 24.52,3.48,Male,No,Sun,Dinner,3 168 | 20.76,2.24,Male,No,Sun,Dinner,2 169 | 31.71,4.5,Male,No,Sun,Dinner,4 170 | 10.59,1.61,Female,Yes,Sat,Dinner,2 171 | 10.63,2.0,Female,Yes,Sat,Dinner,2 172 | 50.81,10.0,Male,Yes,Sat,Dinner,3 173 | 15.81,3.16,Male,Yes,Sat,Dinner,2 174 | 7.25,5.15,Male,Yes,Sun,Dinner,2 175 | 31.85,3.18,Male,Yes,Sun,Dinner,2 176 | 16.82,4.0,Male,Yes,Sun,Dinner,2 177 | 32.9,3.11,Male,Yes,Sun,Dinner,2 178 | 17.89,2.0,Male,Yes,Sun,Dinner,2 179 | 14.48,2.0,Male,Yes,Sun,Dinner,2 180 | 9.6,4.0,Female,Yes,Sun,Dinner,2 181 | 34.63,3.55,Male,Yes,Sun,Dinner,2 182 | 34.65,3.68,Male,Yes,Sun,Dinner,4 183 | 23.33,5.65,Male,Yes,Sun,Dinner,2 184 | 45.35,3.5,Male,Yes,Sun,Dinner,3 185 | 23.17,6.5,Male,Yes,Sun,Dinner,4 186 | 40.55,3.0,Male,Yes,Sun,Dinner,2 187 | 20.69,5.0,Male,No,Sun,Dinner,5 188 | 20.9,3.5,Female,Yes,Sun,Dinner,3 189 | 30.46,2.0,Male,Yes,Sun,Dinner,5 190 | 18.15,3.5,Female,Yes,Sun,Dinner,3 191 | 23.1,4.0,Male,Yes,Sun,Dinner,3 192 | 15.69,1.5,Male,Yes,Sun,Dinner,2 193 | 19.81,4.19,Female,Yes,Thur,Lunch,2 194 | 28.44,2.56,Male,Yes,Thur,Lunch,2 195 | 15.48,2.02,Male,Yes,Thur,Lunch,2 196 | 16.58,4.0,Male,Yes,Thur,Lunch,2 197 | 7.56,1.44,Male,No,Thur,Lunch,2 198 | 10.34,2.0,Male,Yes,Thur,Lunch,2 199 | 43.11,5.0,Female,Yes,Thur,Lunch,4 200 | 13.0,2.0,Female,Yes,Thur,Lunch,2 201 | 13.51,2.0,Male,Yes,Thur,Lunch,2 202 | 18.71,4.0,Male,Yes,Thur,Lunch,3 203 | 12.74,2.01,Female,Yes,Thur,Lunch,2 204 | 13.0,2.0,Female,Yes,Thur,Lunch,2 205 | 16.4,2.5,Female,Yes,Thur,Lunch,2 206 | 20.53,4.0,Male,Yes,Thur,Lunch,4 207 | 16.47,3.23,Female,Yes,Thur,Lunch,3 208 | 26.59,3.41,Male,Yes,Sat,Dinner,3 209 | 38.73,3.0,Male,Yes,Sat,Dinner,4 210 | 24.27,2.03,Male,Yes,Sat,Dinner,2 211 | 12.76,2.23,Female,Yes,Sat,Dinner,2 212 | 30.06,2.0,Male,Yes,Sat,Dinner,3 213 | 25.89,5.16,Male,Yes,Sat,Dinner,4 214 | 48.33,9.0,Male,No,Sat,Dinner,4 215 | 13.27,2.5,Female,Yes,Sat,Dinner,2 216 | 28.17,6.5,Female,Yes,Sat,Dinner,3 217 | 12.9,1.1,Female,Yes,Sat,Dinner,2 218 | 28.15,3.0,Male,Yes,Sat,Dinner,5 219 | 11.59,1.5,Male,Yes,Sat,Dinner,2 220 | 7.74,1.44,Male,Yes,Sat,Dinner,2 221 | 30.14,3.09,Female,Yes,Sat,Dinner,4 222 | 12.16,2.2,Male,Yes,Fri,Lunch,2 223 | 13.42,3.48,Female,Yes,Fri,Lunch,2 224 | 8.58,1.92,Male,Yes,Fri,Lunch,1 225 | 15.98,3.0,Female,No,Fri,Lunch,3 226 | 13.42,1.58,Male,Yes,Fri,Lunch,2 227 | 16.27,2.5,Female,Yes,Fri,Lunch,2 228 | 10.09,2.0,Female,Yes,Fri,Lunch,2 229 | 20.45,3.0,Male,No,Sat,Dinner,4 230 | 13.28,2.72,Male,No,Sat,Dinner,2 231 | 22.12,2.88,Female,Yes,Sat,Dinner,2 232 | 24.01,2.0,Male,Yes,Sat,Dinner,4 233 | 15.69,3.0,Male,Yes,Sat,Dinner,3 234 | 11.61,3.39,Male,No,Sat,Dinner,2 235 | 10.77,1.47,Male,No,Sat,Dinner,2 236 | 15.53,3.0,Male,Yes,Sat,Dinner,2 237 | 10.07,1.25,Male,No,Sat,Dinner,2 238 | 12.6,1.0,Male,Yes,Sat,Dinner,2 239 | 32.83,1.17,Male,Yes,Sat,Dinner,2 240 | 35.83,4.67,Female,No,Sat,Dinner,3 241 | 29.03,5.92,Male,No,Sat,Dinner,3 242 | 27.18,2.0,Female,Yes,Sat,Dinner,2 243 | 22.67,2.0,Male,Yes,Sat,Dinner,2 244 | 17.82,1.75,Male,No,Sat,Dinner,2 245 | 18.78,3.0,Female,No,Thur,Dinner,2 246 | -------------------------------------------------------------------------------- /examples/tips.parquet: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/datafusion-contrib/ray-sql/39a1fef446ecc6c27c1a0ee6039453522d365114/examples/tips.parquet -------------------------------------------------------------------------------- /examples/tips.py: -------------------------------------------------------------------------------- 1 | import os 2 | import pandas as pd 3 | import ray 4 | 5 | from raysql import RaySqlContext 6 | 7 | SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) 8 | 9 | # Start a local cluster 10 | ray.init(resources={"worker": 1}) 11 | 12 | # Create a context and register a table 13 | ctx = RaySqlContext(2, use_ray_shuffle=True) 14 | # Register either a CSV or Parquet file 15 | # ctx.register_csv("tips", f"{SCRIPT_DIR}/tips.csv", True) 16 | ctx.register_parquet("tips", f"{SCRIPT_DIR}/tips.parquet") 17 | 18 | result_set = ctx.sql( 19 | "select sex, smoker, avg(tip/total_bill) as tip_pct from tips group by sex, smoker" 20 | ) 21 | for record_batch in result_set: 22 | print(record_batch.to_pandas()) 23 | -------------------------------------------------------------------------------- /pyproject.toml: -------------------------------------------------------------------------------- 1 | [build-system] 2 | requires = ["maturin>=0.14,<0.15"] 3 | build-backend = "maturin" 4 | 5 | [project] 6 | name = "raysql" 7 | requires-python = ">=3.7" 8 | classifiers = [ 9 | "Programming Language :: Rust", 10 | "Programming Language :: Python :: Implementation :: CPython", 11 | "Programming Language :: Python :: Implementation :: PyPy", 12 | ] 13 | 14 | 15 | -------------------------------------------------------------------------------- /raysql/__init__.py: -------------------------------------------------------------------------------- 1 | try: 2 | import importlib.metadata as importlib_metadata 3 | except ImportError: 4 | import importlib_metadata 5 | 6 | from ._raysql_internal import ( 7 | Context, 8 | ExecutionGraph, 9 | QueryStage, 10 | execute_partition, 11 | serialize_execution_plan, 12 | deserialize_execution_plan, 13 | ) 14 | from .context import RaySqlContext 15 | 16 | __version__ = importlib_metadata.version(__name__) 17 | -------------------------------------------------------------------------------- /raysql/context.py: -------------------------------------------------------------------------------- 1 | import json 2 | import os 3 | import time 4 | from typing import Iterable 5 | 6 | import pyarrow as pa 7 | import ray 8 | 9 | import raysql 10 | from raysql import Context, ExecutionGraph, QueryStage 11 | from typing import List 12 | 13 | def schedule_execution( 14 | graph: ExecutionGraph, 15 | stage_id: int, 16 | is_final_stage: bool, 17 | ) -> list[ray.ObjectRef]: 18 | stage = graph.get_query_stage(stage_id) 19 | # execute child stages first 20 | # A list of (stage ID, list of futures) for each child stage 21 | # Each list is a 2-D array of (input partitions, output partitions). 22 | child_outputs = [] 23 | for child_id in stage.get_child_stage_ids(): 24 | child_outputs.append((child_id, schedule_execution(graph, child_id, False))) 25 | # child_outputs.append((child_id, schedule_execution(graph, child_id))) 26 | 27 | concurrency = stage.get_input_partition_count() 28 | output_partitions_count = stage.get_output_partition_count() 29 | if is_final_stage: 30 | print("Forcing reduce stage concurrency from {} to 1".format(concurrency)) 31 | concurrency = 1 32 | 33 | print( 34 | "Scheduling query stage #{} with {} input partitions and {} output partitions".format( 35 | stage.id(), concurrency, output_partitions_count 36 | ) 37 | ) 38 | 39 | def _get_worker_inputs( 40 | part: int, 41 | ) -> tuple[list[tuple[int, int, int]], list[ray.ObjectRef]]: 42 | ids = [] 43 | futures = [] 44 | for child_stage_id, child_futures in child_outputs: 45 | for i, lst in enumerate(child_futures): 46 | if isinstance(lst, list): 47 | for j, f in enumerate(lst): 48 | if concurrency == 1 or j == part: 49 | # If concurrency is 1, pass in all shuffle partitions. Otherwise, 50 | # only pass in the partitions that match the current worker partition. 51 | ids.append((child_stage_id, i, j)) 52 | futures.append(f) 53 | elif concurrency == 1 or part == 0: 54 | ids.append((child_stage_id, i, 0)) 55 | futures.append(lst) 56 | return ids, futures 57 | 58 | # schedule the actual execution workers 59 | plan_bytes = raysql.serialize_execution_plan(stage.get_execution_plan()) 60 | futures = [] 61 | opt = {} 62 | opt["resources"] = {"worker": 1e-3} 63 | opt["num_returns"] = output_partitions_count 64 | for part in range(concurrency): 65 | ids, inputs = _get_worker_inputs(part) 66 | futures.append( 67 | execute_query_partition.options(**opt).remote( 68 | stage_id, plan_bytes, part, ids, *inputs 69 | ) 70 | ) 71 | return futures 72 | 73 | 74 | @ray.remote(num_cpus=0) 75 | def execute_query_stage( 76 | query_stages: list[QueryStage], 77 | stage_id: int, 78 | use_ray_shuffle: bool, 79 | ) -> tuple[int, list[ray.ObjectRef]]: 80 | """ 81 | Execute a query stage on the workers. 82 | 83 | Returns the stage ID, and a list of futures for the output partitions of the query stage. 84 | """ 85 | stage = QueryStage(stage_id, query_stages[stage_id]) 86 | 87 | # execute child stages first 88 | child_futures = [] 89 | for child_id in stage.get_child_stage_ids(): 90 | child_futures.append( 91 | execute_query_stage.remote(query_stages, child_id, use_ray_shuffle) 92 | ) 93 | 94 | # if the query stage has a single output partition then we need to execute for the output 95 | # partition, otherwise we need to execute in parallel for each input partition 96 | concurrency = stage.get_input_partition_count() 97 | output_partitions_count = stage.get_output_partition_count() 98 | if output_partitions_count == 1: 99 | # reduce stage 100 | print("Forcing reduce stage concurrency from {} to 1".format(concurrency)) 101 | concurrency = 1 102 | 103 | print( 104 | "Scheduling query stage #{} with {} input partitions and {} output partitions".format( 105 | stage.id(), concurrency, output_partitions_count 106 | ) 107 | ) 108 | 109 | # A list of (stage ID, list of futures) for each child stage 110 | # Each list is a 2-D array of (input partitions, output partitions). 111 | child_outputs = ray.get(child_futures) 112 | 113 | def _get_worker_inputs( 114 | part: int, 115 | ) -> tuple[list[tuple[int, int, int]], list[ray.ObjectRef]]: 116 | ids = [] 117 | futures = [] 118 | if use_ray_shuffle: 119 | for child_stage_id, child_futures in child_outputs: 120 | for i, lst in enumerate(child_futures): 121 | if isinstance(lst, list): 122 | for j, f in enumerate(lst): 123 | if concurrency == 1 or j == part: 124 | # If concurrency is 1, pass in all shuffle partitions. Otherwise, 125 | # only pass in the partitions that match the current worker partition. 126 | ids.append((child_stage_id, i, j)) 127 | futures.append(f) 128 | elif concurrency == 1 or part == 0: 129 | ids.append((child_stage_id, i, 0)) 130 | futures.append(lst) 131 | return ids, futures 132 | 133 | # if we are using disk-based shuffle, wait until the child stages to finish 134 | # writing the shuffle files to disk first. 135 | if not use_ray_shuffle: 136 | ray.get([f for _, lst in child_outputs for f in lst]) 137 | 138 | # schedule the actual execution workers 139 | plan_bytes = raysql.serialize_execution_plan(stage.get_execution_plan()) 140 | futures = [] 141 | opt = {} 142 | opt["resources"] = {"worker": 1e-3} 143 | if use_ray_shuffle: 144 | opt["num_returns"] = output_partitions_count 145 | for part in range(concurrency): 146 | ids, inputs = _get_worker_inputs(part) 147 | futures.append( 148 | execute_query_partition.options(**opt).remote( 149 | stage_id, plan_bytes, part, ids, *inputs 150 | ) 151 | ) 152 | 153 | return stage_id, futures 154 | 155 | 156 | @ray.remote 157 | def execute_query_partition( 158 | stage_id: int, 159 | plan_bytes: bytes, 160 | part: int, 161 | input_partition_ids: list[tuple[int, int, int]], 162 | *input_partitions: list[pa.RecordBatch], 163 | ) -> Iterable[pa.RecordBatch]: 164 | start_time = time.time() 165 | plan = raysql.deserialize_execution_plan(plan_bytes) 166 | # print( 167 | # "Worker executing plan {} partition #{} with shuffle inputs {}".format( 168 | # plan.display(), 169 | # part, 170 | # input_partition_ids, 171 | # ) 172 | # ) 173 | partitions = [ 174 | (s, j, p) for (s, _, j), p in zip(input_partition_ids, input_partitions) 175 | ] 176 | # This is delegating to DataFusion for execution, but this would be a good place 177 | # to plug in other execution engines by translating the plan into another engine's plan 178 | # (perhaps via Substrait, once DataFusion supports converting a physical plan to Substrait) 179 | ret = raysql.execute_partition(plan, part, partitions) 180 | duration = time.time() - start_time 181 | event = { 182 | "cat": f"{stage_id}-{part}", 183 | "name": f"{stage_id}-{part}", 184 | "pid": ray.util.get_node_ip_address(), 185 | "tid": os.getpid(), 186 | "ts": int(start_time * 1_000_000), 187 | "dur": int(duration * 1_000_000), 188 | "ph": "X", 189 | } 190 | print(json.dumps(event), end=",") 191 | return ret[0] if len(ret) == 1 else ret 192 | 193 | 194 | class RaySqlContext: 195 | def __init__(self, num_workers: int = 1, use_ray_shuffle: bool = False): 196 | self.ctx = Context(num_workers, use_ray_shuffle) 197 | self.num_workers = num_workers 198 | self.use_ray_shuffle = use_ray_shuffle 199 | 200 | def register_csv(self, table_name: str, path: str, has_header: bool): 201 | self.ctx.register_csv(table_name, path, has_header) 202 | 203 | def register_parquet(self, table_name: str, path: str): 204 | self.ctx.register_parquet(table_name, path) 205 | 206 | def register_data_lake(self, table_name: str, paths: List[str]): 207 | self.ctx.register_datalake_table(table_name, paths) 208 | 209 | def sql(self, sql: str) -> pa.RecordBatch: 210 | # TODO we should parse sql and inspect the plan rather than 211 | # perform a string comparison here 212 | sql_str = sql.lower() 213 | if "create view" in sql_str or "drop view" in sql_str: 214 | self.ctx.sql(sql) 215 | return [] 216 | 217 | graph = self.ctx.plan(sql) 218 | final_stage_id = graph.get_final_query_stage().id() 219 | if self.use_ray_shuffle: 220 | partitions = schedule_execution(graph, final_stage_id, True) 221 | else: 222 | # serialize the query stages and store in Ray object store 223 | query_stages = [ 224 | raysql.serialize_execution_plan( 225 | graph.get_query_stage(i).get_execution_plan() 226 | ) 227 | for i in range(final_stage_id + 1) 228 | ] 229 | # schedule execution 230 | future = execute_query_stage.remote( 231 | query_stages, 232 | final_stage_id, 233 | self.use_ray_shuffle, 234 | ) 235 | _, partitions = ray.get(future) 236 | # assert len(partitions) == 1, len(partitions) 237 | result_set = ray.get(partitions[0]) 238 | return result_set 239 | -------------------------------------------------------------------------------- /raysql/main.py: -------------------------------------------------------------------------------- 1 | import time 2 | import os 3 | 4 | from pyarrow import csv as pacsv 5 | import ray 6 | from raysql import RaySqlContext 7 | 8 | NUM_CPUS_PER_WORKER = 8 9 | 10 | SF = 10 11 | DATA_DIR = f"/mnt/data0/tpch/sf{SF}-parquet" 12 | SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) 13 | QUERIES_DIR = os.path.join(SCRIPT_DIR, f"../sqlbench-h/queries/sf={SF}") 14 | RESULTS_DIR = f"results-sf{SF}" 15 | TRUTH_DIR = ( 16 | "/home/ubuntu/raysort/ray-sql/sqlbench-runners/spark/{RESULTS_DIR}/{RESULTS_DIR}" 17 | ) 18 | 19 | 20 | def setup_context(use_ray_shuffle: bool, num_workers: int = 2) -> RaySqlContext: 21 | print(f"Using {num_workers} workers") 22 | ctx = RaySqlContext(num_workers, use_ray_shuffle) 23 | for table in [ 24 | "customer", 25 | "lineitem", 26 | "nation", 27 | "orders", 28 | "part", 29 | "partsupp", 30 | "region", 31 | "supplier", 32 | ]: 33 | ctx.register_parquet(table, f"{DATA_DIR}/{table}.parquet") 34 | return ctx 35 | 36 | 37 | def load_query(n: int) -> str: 38 | with open(f"{QUERIES_DIR}/q{n}.sql") as fin: 39 | return fin.read() 40 | 41 | 42 | def tpch_query(ctx: RaySqlContext, q: int = 1): 43 | sql = load_query(q) 44 | result_set = ctx.sql(sql) 45 | return result_set 46 | 47 | 48 | def tpch_timing( 49 | ctx: RaySqlContext, 50 | q: int = 1, 51 | print_result: bool = False, 52 | write_result: bool = False, 53 | ): 54 | sql = load_query(q) 55 | start = time.perf_counter() 56 | result = ctx.sql(sql) 57 | end = time.perf_counter() 58 | if print_result: 59 | print("Result:", result) 60 | if isinstance(result, list): 61 | for r in result: 62 | print(r.to_pandas()) 63 | else: 64 | print(result.to_pandas()) 65 | if write_result: 66 | opt = pacsv.WriteOptions(quoting_style="none") 67 | if isinstance(result, list): 68 | for r in result: 69 | pacsv.write_csv(r, f"{RESULTS_DIR}/q{q}.csv", write_options=opt) 70 | else: 71 | pacsv.write_csv(result, f"{RESULTS_DIR}/q{q}.csv", write_options=opt) 72 | return end - start 73 | 74 | 75 | def compare(q: int): 76 | ctx = setup_context(False) 77 | result_set_truth = tpch_query(ctx, q) 78 | 79 | ctx = setup_context(True) 80 | result_set_ray = tpch_query(ctx, q) 81 | 82 | assert result_set_truth == result_set_ray, ( 83 | q, 84 | result_set_truth, 85 | result_set_ray, 86 | ) 87 | 88 | 89 | def tpch_bench(): 90 | ray.init("auto") 91 | num_workers = int(ray.cluster_resources().get("worker", 1)) * NUM_CPUS_PER_WORKER 92 | use_ray_shuffle = False 93 | ctx = setup_context(use_ray_shuffle, num_workers) 94 | # t = tpch_timing(ctx, 11, print_result=True) 95 | # print(f"query,{t},{use_ray_shuffle},{num_workers}") 96 | # return 97 | run_id = time.strftime("%Y-%m-%d-%H-%M-%S") 98 | with open(f"results-sf{SF}-{run_id}.csv", "w") as fout: 99 | for i in range(1, 22 + 1): 100 | if i == 15: 101 | continue 102 | result = tpch_timing(ctx, i, write_result=True) 103 | print(f"query,{i},{result}") 104 | print(f"query,{i},{result}", file=fout, flush=True) 105 | 106 | 107 | tpch_bench() 108 | -------------------------------------------------------------------------------- /raysql/ray_utils.py: -------------------------------------------------------------------------------- 1 | import ray 2 | 3 | 4 | def node_aff(node_id: ray.NodeID, *, soft: bool = False) -> dict: 5 | return { 6 | "scheduling_strategy": ray.util.scheduling_strategies.NodeAffinitySchedulingStrategy( 7 | node_id=node_id, 8 | soft=soft, 9 | ) 10 | } 11 | 12 | 13 | def current_node_aff() -> dict: 14 | return node_aff(ray.get_runtime_context().get_node_id()) 15 | -------------------------------------------------------------------------------- /raysql/tests/test_context.py: -------------------------------------------------------------------------------- 1 | import pytest 2 | from raysql import Context 3 | 4 | def test(): 5 | ctx = Context(1, False) 6 | ctx.register_csv('tips', 'examples/tips.csv', True) 7 | ctx.plan("SELECT * FROM tips") 8 | -------------------------------------------------------------------------------- /requirements-in.txt: -------------------------------------------------------------------------------- 1 | black 2 | flake8 3 | isort 4 | maturin[patchelf] 5 | mypy 6 | numpy 7 | pyarrow 8 | pytest 9 | ray==2.3.0 10 | toml 11 | importlib_metadata; python_version < "3.8" 12 | -------------------------------------------------------------------------------- /src/context.rs: -------------------------------------------------------------------------------- 1 | use crate::planner::{make_execution_graph, PyExecutionGraph}; 2 | use crate::shuffle::{RayShuffleReaderExec, ShuffleCodec}; 3 | use crate::utils::wait_for_future; 4 | use datafusion::arrow::pyarrow::FromPyArrow; 5 | use datafusion::arrow::pyarrow::ToPyArrow; 6 | use datafusion::arrow::record_batch::RecordBatch; 7 | use datafusion::config::Extensions; 8 | use datafusion::error::{DataFusionError, Result}; 9 | use datafusion::execution::context::TaskContext; 10 | use datafusion::execution::disk_manager::DiskManagerConfig; 11 | use datafusion::execution::memory_pool::FairSpillPool; 12 | use datafusion::execution::options::ReadOptions; 13 | use datafusion::execution::runtime_env::RuntimeEnv; 14 | use datafusion::physical_plan::{displayable, ExecutionPlan}; 15 | use datafusion::prelude::*; 16 | use datafusion_proto::bytes::{ 17 | physical_plan_from_bytes_with_extension_codec, physical_plan_to_bytes_with_extension_codec, 18 | }; 19 | use datafusion_python::physical_plan::PyExecutionPlan; 20 | use futures::StreamExt; 21 | use pyo3::prelude::*; 22 | use pyo3::types::{PyList, PyLong, PyTuple}; 23 | use std::collections::HashMap; 24 | use std::sync::Arc; 25 | use tokio::runtime::Runtime; 26 | use tokio::task::JoinHandle; 27 | 28 | type PyResultSet = Vec; 29 | 30 | #[pyclass(name = "Context", module = "raysql", subclass)] 31 | pub struct PyContext { 32 | pub(crate) ctx: SessionContext, 33 | use_ray_shuffle: bool, 34 | } 35 | 36 | #[pymethods] 37 | impl PyContext { 38 | #[new] 39 | pub fn new(target_partitions: usize, use_ray_shuffle: bool) -> Result { 40 | let config = SessionConfig::default() 41 | .with_target_partitions(target_partitions) 42 | .with_batch_size(16 * 1024) 43 | .with_repartition_aggregations(true) 44 | .with_repartition_windows(true) 45 | .with_repartition_joins(true) 46 | .with_parquet_pruning(true); 47 | 48 | let mem_pool_size = 1024 * 1024 * 1024; 49 | let runtime_config = datafusion::execution::runtime_env::RuntimeConfig::new() 50 | .with_memory_pool(Arc::new(FairSpillPool::new(mem_pool_size))) 51 | .with_disk_manager(DiskManagerConfig::new_specified(vec!["/tmp".into()])); 52 | let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); 53 | let ctx = SessionContext::with_config_rt(config, runtime); 54 | Ok(Self { 55 | ctx, 56 | use_ray_shuffle, 57 | }) 58 | } 59 | 60 | pub fn register_csv( 61 | &self, 62 | name: &str, 63 | path: &str, 64 | has_header: bool, 65 | py: Python, 66 | ) -> PyResult<()> { 67 | let options = CsvReadOptions::default().has_header(has_header); 68 | wait_for_future(py, self.ctx.register_csv(name, path, options))?; 69 | Ok(()) 70 | } 71 | 72 | pub fn register_parquet(&self, name: &str, path: &str, py: Python) -> PyResult<()> { 73 | let options = ParquetReadOptions::default(); 74 | wait_for_future(py, self.ctx.register_parquet(name, path, options))?; 75 | Ok(()) 76 | } 77 | 78 | pub fn register_datalake_table(&self, name: &str, path: Vec<&str>, py: Python) -> PyResult<()> { 79 | // let options = ParquetReadOptions::default(); 80 | // let listing_options = options.to_listing_options(&self.ctx.state().config()); 81 | // wait_for_future(py, self.ctx.register_listing_table(name, path, listing_options, None, None))?; 82 | // Ok(()) 83 | unimplemented!() 84 | } 85 | 86 | /// Execute SQL directly against the DataFusion context. Useful for statements 87 | /// such as "create view" or "drop view" 88 | pub fn sql(&self, sql: &str, py: Python) -> PyResult<()> { 89 | println!("Executing {}", sql); 90 | let _df = wait_for_future(py, self.ctx.sql(sql))?; 91 | Ok(()) 92 | } 93 | 94 | /// Plan a distributed SELECT query for executing against the Ray workers 95 | pub fn plan(&self, sql: &str, py: Python) -> PyResult { 96 | println!("Planning {}", sql); 97 | let df = wait_for_future(py, self.ctx.sql(sql))?; 98 | let plan = wait_for_future(py, df.create_physical_plan())?; 99 | 100 | let graph = make_execution_graph(plan.clone(), self.use_ray_shuffle)?; 101 | 102 | // debug logging 103 | let mut stages = graph.query_stages.values().collect::>(); 104 | stages.sort_by_key(|s| s.id); 105 | for stage in stages { 106 | println!( 107 | "Query stage #{}:\n{}", 108 | stage.id, 109 | displayable(stage.plan.as_ref()).indent(false) 110 | ); 111 | } 112 | 113 | Ok(PyExecutionGraph::new(graph)) 114 | } 115 | 116 | /// Execute a partition of a query plan. This will typically be executing a shuffle write and write the results to disk 117 | pub fn execute_partition( 118 | &self, 119 | plan: PyExecutionPlan, 120 | part: usize, 121 | inputs: PyObject, 122 | py: Python, 123 | ) -> PyResultSet { 124 | execute_partition(plan, part, inputs, py) 125 | } 126 | } 127 | 128 | #[pyfunction] 129 | pub fn execute_partition( 130 | plan: PyExecutionPlan, 131 | part: usize, 132 | inputs: PyObject, 133 | py: Python, 134 | ) -> PyResultSet { 135 | _execute_partition(plan, part, inputs) 136 | .unwrap() 137 | .into_iter() 138 | .map(|batch| batch.to_pyarrow(py).unwrap()) // TODO(@lsf) handle error 139 | .collect() 140 | } 141 | 142 | // TODO(@lsf) change this to use pickle 143 | #[pyfunction] 144 | pub fn serialize_execution_plan(plan: PyExecutionPlan) -> PyResult> { 145 | let codec = ShuffleCodec {}; 146 | Ok(physical_plan_to_bytes_with_extension_codec(plan.plan, &codec)?.to_vec()) 147 | } 148 | 149 | #[pyfunction] 150 | pub fn deserialize_execution_plan(bytes: Vec) -> PyResult { 151 | let ctx = SessionContext::new(); 152 | let codec = ShuffleCodec {}; 153 | Ok(PyExecutionPlan::new( 154 | physical_plan_from_bytes_with_extension_codec(&bytes, &ctx, &codec)?, 155 | )) 156 | } 157 | 158 | /// Iterate down an ExecutionPlan and set the input objects for RayShuffleReaderExec. 159 | fn _set_inputs_for_ray_shuffle_reader( 160 | plan: Arc, 161 | part: usize, 162 | input_partitions: &PyList, 163 | ) -> Result<()> { 164 | if let Some(reader_exec) = plan.as_any().downcast_ref::() { 165 | let exec_stage_id = reader_exec.stage_id; 166 | // iterate over inputs, wrap in PyBytes and set as input objects 167 | for item in input_partitions.iter() { 168 | let pytuple = item 169 | .downcast::() 170 | .map_err(|e| DataFusionError::Execution(format!("{}", e)))?; 171 | let stage_id = pytuple 172 | .get_item(0) 173 | .map_err(|e| DataFusionError::Execution(format!("{}", e)))? 174 | .downcast::() 175 | .map_err(|e| DataFusionError::Execution(format!("{}", e)))? 176 | .extract::() 177 | .map_err(|e| DataFusionError::Execution(format!("{}", e)))?; 178 | if stage_id != exec_stage_id { 179 | continue; 180 | } 181 | let part = pytuple 182 | .get_item(1) 183 | .map_err(|e| DataFusionError::Execution(format!("{}", e)))? 184 | .downcast::() 185 | .map_err(|e| DataFusionError::Execution(format!("{}", e)))? 186 | .extract::() 187 | .map_err(|e| DataFusionError::Execution(format!("{}", e)))?; 188 | let batch = RecordBatch::from_pyarrow( 189 | pytuple 190 | .get_item(2) 191 | .map_err(|e| DataFusionError::Execution(format!("{}", e)))?, 192 | ) 193 | .map_err(|e| DataFusionError::Execution(format!("{}", e)))?; 194 | reader_exec.add_input_partition(part, batch)?; 195 | } 196 | } else { 197 | for child in plan.children() { 198 | _set_inputs_for_ray_shuffle_reader(child, part, input_partitions)?; 199 | } 200 | } 201 | Ok(()) 202 | } 203 | 204 | /// Execute a partition of a query plan. This will typically be executing a shuffle write and 205 | /// write the results to disk, except for the final query stage, which will return the data. 206 | /// inputs is a list of tuples of (stage_id, partition_id, bytes) for each input partition. 207 | fn _execute_partition( 208 | plan: PyExecutionPlan, 209 | part: usize, 210 | inputs: PyObject, 211 | ) -> Result> { 212 | let ctx = Arc::new(TaskContext::try_new( 213 | "task_id".to_string(), 214 | "session_id".to_string(), 215 | HashMap::new(), 216 | HashMap::new(), 217 | HashMap::new(), 218 | Arc::new(RuntimeEnv::default()), 219 | Extensions::default(), 220 | )?); 221 | Python::with_gil(|py| { 222 | let input_partitions = inputs 223 | .as_ref(py) 224 | .downcast::() 225 | .map_err(|e| DataFusionError::Execution(format!("{}", e)))?; 226 | _set_inputs_for_ray_shuffle_reader(plan.plan.clone(), part, &input_partitions) 227 | })?; 228 | 229 | // create a Tokio runtime to run the async code 230 | let rt = Runtime::new().unwrap(); 231 | 232 | let fut: JoinHandle>> = rt.spawn(async move { 233 | let mut stream = plan.plan.execute(part, ctx)?; 234 | let mut results = vec![]; 235 | while let Some(result) = stream.next().await { 236 | results.push(result?); 237 | } 238 | Ok(results) 239 | }); 240 | 241 | // block and wait on future 242 | let results = rt.block_on(fut).unwrap()?; 243 | Ok(results) 244 | } 245 | -------------------------------------------------------------------------------- /src/lib.rs: -------------------------------------------------------------------------------- 1 | extern crate core; 2 | 3 | use pyo3::prelude::*; 4 | 5 | mod proto; 6 | use crate::context::{deserialize_execution_plan, execute_partition, serialize_execution_plan}; 7 | pub use proto::generated::protobuf; 8 | 9 | pub mod context; 10 | pub mod planner; 11 | pub mod query_stage; 12 | pub mod shuffle; 13 | pub mod utils; 14 | 15 | /// A Python module implemented in Rust. 16 | #[pymodule] 17 | fn _raysql_internal(_py: Python, m: &PyModule) -> PyResult<()> { 18 | // register classes that can be created directly from Python code 19 | m.add_class::()?; 20 | m.add_class::()?; 21 | m.add_class::()?; 22 | m.add_function(wrap_pyfunction!(execute_partition, m)?)?; 23 | m.add_function(wrap_pyfunction!(serialize_execution_plan, m)?)?; 24 | m.add_function(wrap_pyfunction!(deserialize_execution_plan, m)?)?; 25 | Ok(()) 26 | } 27 | -------------------------------------------------------------------------------- /src/planner.rs: -------------------------------------------------------------------------------- 1 | use crate::query_stage::PyQueryStage; 2 | use crate::query_stage::QueryStage; 3 | use crate::shuffle::{RayShuffleReaderExec, RayShuffleWriterExec}; 4 | use crate::shuffle::{ShuffleReaderExec, ShuffleWriterExec}; 5 | use datafusion::error::Result; 6 | use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; 7 | use datafusion::physical_plan::repartition::RepartitionExec; 8 | use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; 9 | use datafusion::physical_plan::{displayable, Partitioning}; 10 | use datafusion::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; 11 | use log::debug; 12 | use pyo3::prelude::*; 13 | use std::collections::HashMap; 14 | use std::sync::atomic::{AtomicUsize, Ordering}; 15 | use std::sync::Arc; 16 | use uuid::Uuid; 17 | 18 | #[pyclass(name = "ExecutionGraph", module = "raysql", subclass)] 19 | pub struct PyExecutionGraph { 20 | pub graph: ExecutionGraph, 21 | } 22 | 23 | impl PyExecutionGraph { 24 | pub fn new(graph: ExecutionGraph) -> Self { 25 | Self { graph } 26 | } 27 | } 28 | 29 | #[pymethods] 30 | impl PyExecutionGraph { 31 | /// Get a list of stages sorted by id 32 | pub fn get_query_stages(&self) -> Vec { 33 | let mut stages = vec![]; 34 | let max_id = self.graph.get_final_query_stage().id; 35 | for id in 0..=max_id { 36 | stages.push(PyQueryStage::from_rust( 37 | self.graph.query_stages.get(&id).unwrap().clone(), 38 | )); 39 | } 40 | stages 41 | } 42 | 43 | pub fn get_query_stage(&self, id: usize) -> PyResult { 44 | if let Some(stage) = self.graph.query_stages.get(&id) { 45 | Ok(PyQueryStage::from_rust(stage.clone())) 46 | } else { 47 | todo!() 48 | } 49 | } 50 | 51 | pub fn get_final_query_stage(&self) -> PyQueryStage { 52 | PyQueryStage::from_rust(self.graph.get_final_query_stage()) 53 | } 54 | } 55 | 56 | #[derive(Debug)] 57 | pub struct ExecutionGraph { 58 | /// Query stages by id 59 | pub query_stages: HashMap>, 60 | id_generator: AtomicUsize, 61 | } 62 | 63 | impl Default for ExecutionGraph { 64 | fn default() -> Self { 65 | Self::new() 66 | } 67 | } 68 | 69 | impl ExecutionGraph { 70 | pub fn new() -> Self { 71 | Self { 72 | query_stages: HashMap::new(), 73 | id_generator: AtomicUsize::new(0), 74 | } 75 | } 76 | 77 | fn add_query_stage(&mut self, stage_id: usize, plan: Arc) -> usize { 78 | let query_stage = QueryStage::new(stage_id, plan); 79 | self.query_stages.insert(stage_id, Arc::new(query_stage)); 80 | stage_id 81 | } 82 | 83 | fn get_final_query_stage(&self) -> Arc { 84 | // the final query stage is always the last to be created and 85 | // therefore has the highest id 86 | let mut max_id = 0; 87 | for k in self.query_stages.keys() { 88 | if *k > max_id { 89 | max_id = *k; 90 | } 91 | } 92 | self.query_stages.get(&max_id).unwrap().clone() 93 | } 94 | 95 | fn next_id(&self) -> usize { 96 | self.id_generator.fetch_add(1, Ordering::Relaxed) 97 | } 98 | } 99 | 100 | pub fn make_execution_graph( 101 | plan: Arc, 102 | use_ray_shuffle: bool, 103 | ) -> Result { 104 | let mut graph = ExecutionGraph::new(); 105 | let root = generate_query_stages(plan, &mut graph, use_ray_shuffle)?; 106 | // We force the final stage to produce a single partition to return 107 | // to the driver. This might not suit ETL workloads. 108 | if root.output_partitioning().partition_count() > 1 { 109 | let root = Arc::new(CoalescePartitionsExec::new(root)); 110 | graph.add_query_stage(graph.next_id(), root); 111 | } else { 112 | graph.add_query_stage(graph.next_id(), root); 113 | } 114 | Ok(graph) 115 | } 116 | 117 | /// Convert a physical query plan into a distributed physical query plan by breaking the query 118 | /// into query stages based on changes in partitioning. 119 | fn generate_query_stages( 120 | plan: Arc, 121 | graph: &mut ExecutionGraph, 122 | use_ray_shuffle: bool, 123 | ) -> Result> { 124 | // recurse down first 125 | let new_children: Vec> = plan 126 | .children() 127 | .iter() 128 | .map(|x| generate_query_stages(x.clone(), graph, use_ray_shuffle)) 129 | .collect::>>()?; 130 | let plan = with_new_children_if_necessary(plan, new_children)?.into(); 131 | 132 | debug!("plan = {}", displayable(plan.as_ref()).one_line()); 133 | debug!("output_part = {:?}", plan.output_partitioning()); 134 | 135 | let new_plan = if let Some(repart) = plan.as_any().downcast_ref::() { 136 | match repart.partitioning() { 137 | &Partitioning::UnknownPartitioning(_) | &Partitioning::RoundRobinBatch(_) => { 138 | // just remove these 139 | Ok(repart.children()[0].clone()) 140 | } 141 | partitioning_scheme => create_shuffle_exchange( 142 | plan.children()[0].clone(), 143 | graph, 144 | partitioning_scheme.clone(), 145 | use_ray_shuffle, 146 | ), 147 | } 148 | } else if plan 149 | .as_any() 150 | .downcast_ref::() 151 | .is_some() 152 | { 153 | let coalesce_input = plan.children()[0].clone(); 154 | let partitioning_scheme = coalesce_input.output_partitioning(); 155 | let new_input = 156 | create_shuffle_exchange(coalesce_input, graph, partitioning_scheme, use_ray_shuffle)?; 157 | with_new_children_if_necessary(plan, vec![new_input]).map(|p| p.into()) 158 | } else if plan 159 | .as_any() 160 | .downcast_ref::() 161 | .is_some() 162 | { 163 | let partitioned_sort_plan = plan.children()[0].clone(); 164 | let partitioning_scheme = partitioned_sort_plan.output_partitioning(); 165 | let new_input = create_shuffle_exchange( 166 | partitioned_sort_plan, 167 | graph, 168 | partitioning_scheme, 169 | use_ray_shuffle, 170 | )?; 171 | with_new_children_if_necessary(plan, vec![new_input]).map(|p| p.into()) 172 | } else { 173 | Ok(plan) 174 | }?; 175 | 176 | debug!("new_plan = {}", displayable(new_plan.as_ref()).one_line()); 177 | debug!( 178 | "new_output_part = {:?}\n\n-------------------------\n\n", 179 | new_plan.output_partitioning() 180 | ); 181 | 182 | Ok(new_plan) 183 | } 184 | 185 | /// Create a shuffle exchange. 186 | /// 187 | /// The plan is wrapped in a ShuffleWriteExec and added as a new query plan in the execution graph 188 | /// and a ShuffleReaderExec is returned to replace the plan. 189 | fn create_shuffle_exchange( 190 | plan: Arc, 191 | graph: &mut ExecutionGraph, 192 | partitioning_scheme: Partitioning, 193 | use_ray_shuffle: bool, 194 | ) -> Result> { 195 | // introduce shuffle to produce one output partition 196 | let stage_id = graph.next_id(); 197 | 198 | // create temp dir for stage shuffle files 199 | let temp_dir = create_temp_dir(stage_id)?; 200 | 201 | let shuffle_writer_input = plan.clone(); 202 | let shuffle_writer: Arc = if use_ray_shuffle { 203 | Arc::new(RayShuffleWriterExec::new( 204 | stage_id, 205 | shuffle_writer_input, 206 | partitioning_scheme.clone(), 207 | )) 208 | } else { 209 | Arc::new(ShuffleWriterExec::new( 210 | stage_id, 211 | shuffle_writer_input, 212 | partitioning_scheme.clone(), 213 | &temp_dir, 214 | )) 215 | }; 216 | 217 | debug!( 218 | "Created shuffle writer with output partitioning {:?}", 219 | shuffle_writer.output_partitioning() 220 | ); 221 | 222 | let stage_id = graph.add_query_stage(stage_id, shuffle_writer); 223 | // replace the plan with a shuffle reader 224 | if use_ray_shuffle { 225 | Ok(Arc::new(RayShuffleReaderExec::new( 226 | stage_id, 227 | plan.schema(), 228 | partitioning_scheme, 229 | ))) 230 | } else { 231 | Ok(Arc::new(ShuffleReaderExec::new( 232 | stage_id, 233 | plan.schema(), 234 | partitioning_scheme, 235 | &temp_dir, 236 | ))) 237 | } 238 | } 239 | 240 | fn create_temp_dir(stage_id: usize) -> Result { 241 | let uuid = Uuid::new_v4(); 242 | let temp_dir = format!("/tmp/ray-sql-{uuid}-stage-{stage_id}"); 243 | debug!("Creating temp shuffle dir: {temp_dir}"); 244 | std::fs::create_dir(&temp_dir)?; 245 | Ok(temp_dir) 246 | } 247 | 248 | #[cfg(test)] 249 | mod test { 250 | use super::*; 251 | use datafusion::physical_plan::displayable; 252 | use datafusion::prelude::{ParquetReadOptions, SessionConfig, SessionContext}; 253 | use std::fs; 254 | use std::path::Path; 255 | 256 | #[tokio::test] 257 | async fn test_q1() -> Result<()> { 258 | do_test(1).await 259 | } 260 | 261 | #[tokio::test] 262 | async fn test_q2() -> Result<()> { 263 | do_test(2).await 264 | } 265 | 266 | #[tokio::test] 267 | async fn test_q3() -> Result<()> { 268 | do_test(3).await 269 | } 270 | 271 | #[tokio::test] 272 | async fn test_q4() -> Result<()> { 273 | do_test(4).await 274 | } 275 | 276 | #[tokio::test] 277 | async fn test_q5() -> Result<()> { 278 | do_test(5).await 279 | } 280 | 281 | #[tokio::test] 282 | async fn test_q6() -> Result<()> { 283 | do_test(6).await 284 | } 285 | 286 | #[tokio::test] 287 | async fn test_q7() -> Result<()> { 288 | do_test(7).await 289 | } 290 | 291 | #[tokio::test] 292 | async fn test_q8() -> Result<()> { 293 | do_test(8).await 294 | } 295 | 296 | #[tokio::test] 297 | async fn test_q9() -> Result<()> { 298 | do_test(9).await 299 | } 300 | 301 | #[tokio::test] 302 | async fn test_q10() -> Result<()> { 303 | do_test(10).await 304 | } 305 | 306 | #[tokio::test] 307 | async fn test_q11() -> Result<()> { 308 | do_test(11).await 309 | } 310 | 311 | #[tokio::test] 312 | async fn test_q12() -> Result<()> { 313 | do_test(12).await 314 | } 315 | 316 | #[tokio::test] 317 | async fn test_q13() -> Result<()> { 318 | do_test(13).await 319 | } 320 | 321 | #[tokio::test] 322 | async fn test_q14() -> Result<()> { 323 | do_test(14).await 324 | } 325 | 326 | #[ignore] 327 | #[tokio::test] 328 | async fn test_q15() -> Result<()> { 329 | do_test(15).await 330 | } 331 | 332 | #[tokio::test] 333 | async fn test_q16() -> Result<()> { 334 | do_test(16).await 335 | } 336 | 337 | #[tokio::test] 338 | async fn test_q17() -> Result<()> { 339 | do_test(17).await 340 | } 341 | 342 | #[tokio::test] 343 | async fn test_q18() -> Result<()> { 344 | do_test(18).await 345 | } 346 | 347 | #[tokio::test] 348 | async fn test_q19() -> Result<()> { 349 | do_test(19).await 350 | } 351 | 352 | #[tokio::test] 353 | async fn test_q20() -> Result<()> { 354 | do_test(20).await 355 | } 356 | 357 | #[tokio::test] 358 | async fn test_q21() -> Result<()> { 359 | do_test(21).await 360 | } 361 | 362 | #[tokio::test] 363 | async fn test_q22() -> Result<()> { 364 | do_test(22).await 365 | } 366 | 367 | async fn do_test(n: u8) -> Result<()> { 368 | let data_path = "/mnt/bigdata/tpch/sf10-parquet"; 369 | if !Path::new(&data_path).exists() { 370 | return Ok(()); 371 | } 372 | let file = format!("testdata/queries/q{n}.sql"); 373 | let sql = fs::read_to_string(&file)?; 374 | let config = SessionConfig::new().with_target_partitions(4); 375 | let ctx = SessionContext::with_config(config); 376 | let tables = &[ 377 | "customer", "lineitem", "nation", "orders", "part", "partsupp", "region", "supplier", 378 | ]; 379 | for table in tables { 380 | ctx.register_parquet( 381 | table, 382 | &format!("{data_path}/{table}.parquet"), 383 | ParquetReadOptions::default(), 384 | ) 385 | .await?; 386 | } 387 | let mut output = String::new(); 388 | 389 | let df = ctx.sql(&sql).await?; 390 | 391 | let plan = df.clone().into_optimized_plan()?; 392 | output.push_str(&format!( 393 | "DataFusion Logical Plan\n=======================\n\n{}\n\n", 394 | plan.display_indent() 395 | )); 396 | 397 | let plan = df.create_physical_plan().await?; 398 | output.push_str(&format!( 399 | "DataFusion Physical Plan\n========================\n\n{}\n", 400 | displayable(plan.as_ref()).indent(false) 401 | )); 402 | 403 | output.push_str("RaySQL Plan\n===========\n\n"); 404 | let graph = make_execution_graph(plan, false)?; 405 | for id in 0..=graph.get_final_query_stage().id { 406 | let query_stage = graph.query_stages.get(&id).unwrap(); 407 | output.push_str(&format!( 408 | "Query Stage #{id} ({} -> {}):\n{}\n", 409 | query_stage.get_input_partition_count(), 410 | query_stage.get_output_partition_count(), 411 | displayable(query_stage.plan.as_ref()).indent(false) 412 | )); 413 | } 414 | let expected_file = format!("testdata/expected-plans/q{n}.txt"); 415 | if !Path::new(&expected_file).exists() { 416 | fs::write(&expected_file, &output)?; 417 | } 418 | let expected_plan = fs::read_to_string(&expected_file)?; 419 | assert_eq!(expected_plan, output); 420 | Ok(()) 421 | } 422 | } 423 | -------------------------------------------------------------------------------- /src/proto/generated/mod.rs: -------------------------------------------------------------------------------- 1 | pub mod protobuf; 2 | -------------------------------------------------------------------------------- /src/proto/generated/protobuf.rs: -------------------------------------------------------------------------------- 1 | #[allow(clippy::derive_partial_eq_without_eq)] 2 | #[derive(Clone, PartialEq, ::prost::Message)] 3 | pub struct RaySqlExecNode { 4 | #[prost(oneof = "ray_sql_exec_node::PlanType", tags = "1, 2, 3, 4")] 5 | pub plan_type: ::core::option::Option, 6 | } 7 | /// Nested message and enum types in `RaySqlExecNode`. 8 | pub mod ray_sql_exec_node { 9 | #[allow(clippy::derive_partial_eq_without_eq)] 10 | #[derive(Clone, PartialEq, ::prost::Oneof)] 11 | pub enum PlanType { 12 | #[prost(message, tag = "1")] 13 | ShuffleReader(super::ShuffleReaderExecNode), 14 | #[prost(message, tag = "2")] 15 | ShuffleWriter(super::ShuffleWriterExecNode), 16 | #[prost(message, tag = "3")] 17 | RayShuffleReader(super::RayShuffleReaderExecNode), 18 | #[prost(message, tag = "4")] 19 | RayShuffleWriter(super::RayShuffleWriterExecNode), 20 | } 21 | } 22 | #[allow(clippy::derive_partial_eq_without_eq)] 23 | #[derive(Clone, PartialEq, ::prost::Message)] 24 | pub struct ShuffleReaderExecNode { 25 | /// stage to read from 26 | #[prost(uint32, tag = "1")] 27 | pub stage_id: u32, 28 | /// schema of the shuffle stage 29 | #[prost(message, optional, tag = "2")] 30 | pub schema: ::core::option::Option<::datafusion_proto::protobuf::Schema>, 31 | /// this must match the output partitioning of the writer we are reading from 32 | #[prost(message, optional, tag = "3")] 33 | pub partitioning: ::core::option::Option< 34 | ::datafusion_proto::protobuf::PhysicalHashRepartition, 35 | >, 36 | /// directory for shuffle files 37 | #[prost(string, tag = "4")] 38 | pub shuffle_dir: ::prost::alloc::string::String, 39 | } 40 | #[allow(clippy::derive_partial_eq_without_eq)] 41 | #[derive(Clone, PartialEq, ::prost::Message)] 42 | pub struct ShuffleWriterExecNode { 43 | /// stage that is writing the shuffle files 44 | #[prost(uint32, tag = "1")] 45 | pub stage_id: u32, 46 | /// plan to execute 47 | #[prost(message, optional, tag = "2")] 48 | pub plan: ::core::option::Option<::datafusion_proto::protobuf::PhysicalPlanNode>, 49 | /// output partitioning schema 50 | #[prost(message, optional, tag = "3")] 51 | pub partitioning: ::core::option::Option< 52 | ::datafusion_proto::protobuf::PhysicalHashRepartition, 53 | >, 54 | /// directory for shuffle files 55 | #[prost(string, tag = "4")] 56 | pub shuffle_dir: ::prost::alloc::string::String, 57 | } 58 | #[allow(clippy::derive_partial_eq_without_eq)] 59 | #[derive(Clone, PartialEq, ::prost::Message)] 60 | pub struct RayShuffleReaderExecNode { 61 | /// stage to read from 62 | #[prost(uint32, tag = "1")] 63 | pub stage_id: u32, 64 | /// schema of the shuffle stage 65 | #[prost(message, optional, tag = "2")] 66 | pub schema: ::core::option::Option<::datafusion_proto::protobuf::Schema>, 67 | /// this must match the output partitioning of the writer we are reading from 68 | #[prost(message, optional, tag = "3")] 69 | pub partitioning: ::core::option::Option< 70 | ::datafusion_proto::protobuf::PhysicalHashRepartition, 71 | >, 72 | } 73 | #[allow(clippy::derive_partial_eq_without_eq)] 74 | #[derive(Clone, PartialEq, ::prost::Message)] 75 | pub struct RayShuffleWriterExecNode { 76 | /// stage that is writing the shuffle files 77 | #[prost(uint32, tag = "1")] 78 | pub stage_id: u32, 79 | /// plan to execute 80 | #[prost(message, optional, tag = "2")] 81 | pub plan: ::core::option::Option<::datafusion_proto::protobuf::PhysicalPlanNode>, 82 | /// output partitioning schema 83 | #[prost(message, optional, tag = "3")] 84 | pub partitioning: ::core::option::Option< 85 | ::datafusion_proto::protobuf::PhysicalHashRepartition, 86 | >, 87 | } 88 | -------------------------------------------------------------------------------- /src/proto/mod.rs: -------------------------------------------------------------------------------- 1 | pub mod generated; 2 | -------------------------------------------------------------------------------- /src/proto/raysql.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | 3 | package raysql.protobuf; 4 | 5 | option java_multiple_files = true; 6 | option java_package = "raysql.protobuf"; 7 | option java_outer_classname = "RaySqlProto"; 8 | 9 | import "datafusion.proto"; 10 | 11 | message RaySqlExecNode { 12 | oneof PlanType { 13 | ShuffleReaderExecNode shuffle_reader = 1; 14 | ShuffleWriterExecNode shuffle_writer = 2; 15 | RayShuffleReaderExecNode ray_shuffle_reader = 3; 16 | RayShuffleWriterExecNode ray_shuffle_writer = 4; 17 | } 18 | } 19 | 20 | message ShuffleReaderExecNode { 21 | // stage to read from 22 | uint32 stage_id = 1; 23 | // schema of the shuffle stage 24 | datafusion.Schema schema = 2; 25 | // this must match the output partitioning of the writer we are reading from 26 | datafusion.PhysicalHashRepartition partitioning = 3; 27 | // directory for shuffle files 28 | string shuffle_dir = 4; 29 | } 30 | 31 | message ShuffleWriterExecNode { 32 | // stage that is writing the shuffle files 33 | uint32 stage_id = 1; 34 | // plan to execute 35 | datafusion.PhysicalPlanNode plan = 2; 36 | // output partitioning schema 37 | datafusion.PhysicalHashRepartition partitioning = 3; 38 | // directory for shuffle files 39 | string shuffle_dir = 4; 40 | } 41 | 42 | message RayShuffleReaderExecNode { 43 | // stage to read from 44 | uint32 stage_id = 1; 45 | // schema of the shuffle stage 46 | datafusion.Schema schema = 2; 47 | // this must match the output partitioning of the writer we are reading from 48 | datafusion.PhysicalHashRepartition partitioning = 3; 49 | } 50 | 51 | message RayShuffleWriterExecNode { 52 | // stage that is writing the shuffle files 53 | uint32 stage_id = 1; 54 | // plan to execute 55 | datafusion.PhysicalPlanNode plan = 2; 56 | // output partitioning schema 57 | datafusion.PhysicalHashRepartition partitioning = 3; 58 | } 59 | -------------------------------------------------------------------------------- /src/query_stage.rs: -------------------------------------------------------------------------------- 1 | use crate::shuffle::{RayShuffleReaderExec, ShuffleCodec, ShuffleReaderExec}; 2 | use datafusion::error::Result; 3 | use datafusion::physical_plan::{ExecutionPlan, Partitioning}; 4 | use datafusion::prelude::SessionContext; 5 | use datafusion_proto::bytes::physical_plan_from_bytes_with_extension_codec; 6 | use datafusion_python::physical_plan::PyExecutionPlan; 7 | use pyo3::prelude::*; 8 | use std::sync::Arc; 9 | 10 | #[pyclass(name = "QueryStage", module = "raysql", subclass)] 11 | pub struct PyQueryStage { 12 | stage: Arc, 13 | } 14 | 15 | impl PyQueryStage { 16 | pub fn from_rust(stage: Arc) -> Self { 17 | Self { stage } 18 | } 19 | } 20 | 21 | #[pymethods] 22 | impl PyQueryStage { 23 | #[new] 24 | pub fn new(id: usize, bytes: Vec) -> Result { 25 | let ctx = SessionContext::new(); 26 | let codec = ShuffleCodec {}; 27 | let plan = physical_plan_from_bytes_with_extension_codec(&bytes, &ctx, &codec)?; 28 | Ok(PyQueryStage { 29 | stage: Arc::new(QueryStage { id, plan }), 30 | }) 31 | } 32 | 33 | pub fn id(&self) -> usize { 34 | self.stage.id 35 | } 36 | 37 | pub fn get_execution_plan(&self) -> PyExecutionPlan { 38 | PyExecutionPlan::new(self.stage.plan.clone()) 39 | } 40 | 41 | pub fn get_child_stage_ids(&self) -> Vec { 42 | self.stage.get_child_stage_ids() 43 | } 44 | 45 | pub fn get_input_partition_count(&self) -> usize { 46 | self.stage.get_input_partition_count() 47 | } 48 | 49 | pub fn get_output_partition_count(&self) -> usize { 50 | self.stage.get_output_partition_count() 51 | } 52 | } 53 | 54 | #[derive(Debug)] 55 | pub struct QueryStage { 56 | pub id: usize, 57 | pub plan: Arc, 58 | } 59 | 60 | fn _get_output_partition_count(plan: &dyn ExecutionPlan) -> usize { 61 | // UnknownPartitioning and HashPartitioning with empty expressions will 62 | // both return 1 partition. 63 | match plan.output_partitioning() { 64 | Partitioning::UnknownPartitioning(_) => 1, 65 | Partitioning::Hash(expr, _) if expr.is_empty() => 1, 66 | p => p.partition_count(), 67 | } 68 | } 69 | 70 | impl QueryStage { 71 | pub fn new(id: usize, plan: Arc) -> Self { 72 | Self { id, plan } 73 | } 74 | 75 | pub fn get_child_stage_ids(&self) -> Vec { 76 | let mut ids = vec![]; 77 | collect_child_stage_ids(self.plan.as_ref(), &mut ids); 78 | ids 79 | } 80 | 81 | /// Get the input partition count. This is the same as the number of concurrent tasks 82 | /// when we schedule this query stage for execution 83 | pub fn get_input_partition_count(&self) -> usize { 84 | self.plan.children()[0] 85 | .output_partitioning() 86 | .partition_count() 87 | } 88 | 89 | pub fn get_output_partition_count(&self) -> usize { 90 | _get_output_partition_count(self.plan.as_ref()) 91 | } 92 | } 93 | 94 | fn collect_child_stage_ids(plan: &dyn ExecutionPlan, ids: &mut Vec) { 95 | if let Some(shuffle_reader) = plan.as_any().downcast_ref::() { 96 | ids.push(shuffle_reader.stage_id); 97 | } else if let Some(shuffle_reader) = plan.as_any().downcast_ref::() { 98 | ids.push(shuffle_reader.stage_id); 99 | } else { 100 | for child_plan in plan.children() { 101 | collect_child_stage_ids(child_plan.as_ref(), ids); 102 | } 103 | } 104 | } 105 | -------------------------------------------------------------------------------- /src/shuffle/codec.rs: -------------------------------------------------------------------------------- 1 | use crate::protobuf::ray_sql_exec_node::PlanType; 2 | use crate::protobuf::{ 3 | RayShuffleReaderExecNode, RayShuffleWriterExecNode, RaySqlExecNode, ShuffleReaderExecNode, 4 | ShuffleWriterExecNode, 5 | }; 6 | use crate::shuffle::{ 7 | RayShuffleReaderExec, RayShuffleWriterExec, ShuffleReaderExec, ShuffleWriterExec, 8 | }; 9 | use datafusion::arrow::datatypes::SchemaRef; 10 | use datafusion::common::{DataFusionError, Result}; 11 | use datafusion::execution::runtime_env::RuntimeEnv; 12 | use datafusion::execution::FunctionRegistry; 13 | use datafusion::logical_expr::{AggregateUDF, ScalarUDF, WindowUDF}; 14 | use datafusion::physical_plan::{ExecutionPlan, Partitioning}; 15 | use datafusion_proto::physical_plan::from_proto::parse_protobuf_hash_partitioning; 16 | use datafusion_proto::physical_plan::AsExecutionPlan; 17 | use datafusion_proto::physical_plan::PhysicalExtensionCodec; 18 | use datafusion_proto::protobuf; 19 | use datafusion_proto::protobuf::{PhysicalHashRepartition, PhysicalPlanNode}; 20 | use prost::Message; 21 | use std::collections::HashSet; 22 | use std::sync::Arc; 23 | 24 | #[derive(Debug)] 25 | pub struct ShuffleCodec {} 26 | 27 | impl PhysicalExtensionCodec for ShuffleCodec { 28 | fn try_decode( 29 | &self, 30 | buf: &[u8], 31 | _inputs: &[Arc], 32 | registry: &dyn FunctionRegistry, 33 | ) -> Result, DataFusionError> { 34 | // decode bytes to protobuf struct 35 | let node = RaySqlExecNode::decode(buf) 36 | .map_err(|e| DataFusionError::Internal(format!("failed to decode plan: {e:?}")))?; 37 | match node.plan_type { 38 | Some(PlanType::ShuffleReader(reader)) => { 39 | let schema = reader.schema.as_ref().unwrap(); 40 | let schema: SchemaRef = Arc::new(schema.try_into().unwrap()); 41 | let hash_part = parse_protobuf_hash_partitioning( 42 | reader.partitioning.as_ref(), 43 | registry, 44 | &schema, 45 | )?; 46 | Ok(Arc::new(ShuffleReaderExec::new( 47 | reader.stage_id as usize, 48 | schema, 49 | hash_part.unwrap(), 50 | &reader.shuffle_dir, 51 | ))) 52 | } 53 | Some(PlanType::ShuffleWriter(writer)) => { 54 | let plan = writer.plan.unwrap().try_into_physical_plan( 55 | registry, 56 | &RuntimeEnv::default(), 57 | self, 58 | )?; 59 | let hash_part = parse_protobuf_hash_partitioning( 60 | writer.partitioning.as_ref(), 61 | registry, 62 | plan.schema().as_ref(), 63 | )?; 64 | Ok(Arc::new(ShuffleWriterExec::new( 65 | writer.stage_id as usize, 66 | plan, 67 | hash_part.unwrap(), 68 | &writer.shuffle_dir, 69 | ))) 70 | } 71 | Some(PlanType::RayShuffleReader(reader)) => { 72 | let schema = reader.schema.as_ref().unwrap(); 73 | let schema: SchemaRef = Arc::new(schema.try_into().unwrap()); 74 | let hash_part = parse_protobuf_hash_partitioning( 75 | reader.partitioning.as_ref(), 76 | registry, 77 | &schema, 78 | )?; 79 | Ok(Arc::new(RayShuffleReaderExec::new( 80 | reader.stage_id as usize, 81 | schema, 82 | hash_part.unwrap(), 83 | ))) 84 | } 85 | Some(PlanType::RayShuffleWriter(writer)) => { 86 | let plan = writer.plan.unwrap().try_into_physical_plan( 87 | registry, 88 | &RuntimeEnv::default(), 89 | self, 90 | )?; 91 | let hash_part = parse_protobuf_hash_partitioning( 92 | writer.partitioning.as_ref(), 93 | registry, 94 | plan.schema().as_ref(), 95 | )?; 96 | Ok(Arc::new(RayShuffleWriterExec::new( 97 | writer.stage_id as usize, 98 | plan, 99 | hash_part.unwrap(), 100 | ))) 101 | } 102 | _ => unreachable!(), 103 | } 104 | } 105 | 106 | fn try_encode( 107 | &self, 108 | node: Arc, 109 | buf: &mut Vec, 110 | ) -> Result<(), DataFusionError> { 111 | let plan = if let Some(reader) = node.as_any().downcast_ref::() { 112 | let schema: protobuf::Schema = reader.schema().try_into().unwrap(); 113 | let partitioning = encode_partitioning_scheme(&reader.output_partitioning())?; 114 | let reader = ShuffleReaderExecNode { 115 | stage_id: reader.stage_id as u32, 116 | schema: Some(schema), 117 | partitioning: Some(partitioning), 118 | shuffle_dir: reader.shuffle_dir.clone(), 119 | }; 120 | PlanType::ShuffleReader(reader) 121 | } else if let Some(writer) = node.as_any().downcast_ref::() { 122 | let plan = PhysicalPlanNode::try_from_physical_plan(writer.plan.clone(), self)?; 123 | let partitioning = encode_partitioning_scheme(&writer.output_partitioning())?; 124 | let writer = ShuffleWriterExecNode { 125 | stage_id: writer.stage_id as u32, 126 | plan: Some(plan), 127 | partitioning: Some(partitioning), 128 | shuffle_dir: writer.shuffle_dir.clone(), 129 | }; 130 | PlanType::ShuffleWriter(writer) 131 | } else if let Some(reader) = node.as_any().downcast_ref::() { 132 | let schema: protobuf::Schema = reader.schema().try_into().unwrap(); 133 | let partitioning = encode_partitioning_scheme(&reader.output_partitioning())?; 134 | let reader = RayShuffleReaderExecNode { 135 | stage_id: reader.stage_id as u32, 136 | schema: Some(schema), 137 | partitioning: Some(partitioning), 138 | }; 139 | PlanType::RayShuffleReader(reader) 140 | } else if let Some(writer) = node.as_any().downcast_ref::() { 141 | let plan = PhysicalPlanNode::try_from_physical_plan(writer.plan.clone(), self)?; 142 | let partitioning = encode_partitioning_scheme(&writer.output_partitioning())?; 143 | let writer = RayShuffleWriterExecNode { 144 | stage_id: writer.stage_id as u32, 145 | plan: Some(plan), 146 | partitioning: Some(partitioning), 147 | }; 148 | PlanType::RayShuffleWriter(writer) 149 | } else { 150 | unreachable!() 151 | }; 152 | plan.encode(buf); 153 | Ok(()) 154 | } 155 | } 156 | 157 | fn encode_partitioning_scheme(partitioning: &Partitioning) -> Result { 158 | match partitioning { 159 | Partitioning::Hash(expr, partition_count) => Ok(protobuf::PhysicalHashRepartition { 160 | hash_expr: expr 161 | .iter() 162 | .map(|expr| expr.clone().try_into()) 163 | .collect::, DataFusionError>>()?, 164 | partition_count: *partition_count as u64, 165 | }), 166 | Partitioning::UnknownPartitioning(n) => Ok(protobuf::PhysicalHashRepartition { 167 | hash_expr: vec![], 168 | partition_count: *n as u64, 169 | }), 170 | other => Err(DataFusionError::Plan(format!( 171 | "Unsupported shuffle partitioning scheme: {other:?}" 172 | ))), 173 | } 174 | } 175 | 176 | struct RaySqlFunctionRegistry {} 177 | 178 | impl FunctionRegistry for RaySqlFunctionRegistry { 179 | fn udfs(&self) -> HashSet { 180 | HashSet::new() 181 | } 182 | 183 | fn udf(&self, name: &str) -> datafusion::common::Result> { 184 | Err(DataFusionError::Plan(format!("Invalid UDF: {name}"))) 185 | } 186 | 187 | fn udaf(&self, name: &str) -> datafusion::common::Result> { 188 | Err(DataFusionError::Plan(format!("Invalid UDAF: {name}"))) 189 | } 190 | 191 | fn udwf(&self, name: &str) -> datafusion::common::Result> { 192 | Err(DataFusionError::Plan(format!("Invalid UDAWF: {name}"))) 193 | } 194 | } 195 | -------------------------------------------------------------------------------- /src/shuffle/mod.rs: -------------------------------------------------------------------------------- 1 | mod codec; 2 | mod ray_shuffle; 3 | mod reader; 4 | mod writer; 5 | 6 | pub use codec::ShuffleCodec; 7 | pub use ray_shuffle::RayShuffleReaderExec; 8 | pub use ray_shuffle::RayShuffleWriterExec; 9 | pub use reader::ShuffleReaderExec; 10 | pub use writer::ShuffleWriterExec; 11 | -------------------------------------------------------------------------------- /src/shuffle/ray_shuffle/mod.rs: -------------------------------------------------------------------------------- 1 | mod reader; 2 | mod writer; 3 | 4 | use arrow::record_batch::RecordBatch; 5 | use datafusion::arrow; 6 | use datafusion::arrow::datatypes::SchemaRef; 7 | use datafusion::common::Result; 8 | use datafusion::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; 9 | use futures::Stream; 10 | pub use reader::RayShuffleReaderExec; 11 | use std::pin::Pin; 12 | use std::task::{Context, Poll}; 13 | use tokio::macros::support::thread_rng_n; 14 | pub use writer::RayShuffleWriterExec; 15 | 16 | /// CombinedRecordBatchStream can be used to combine a Vec of SendableRecordBatchStreams into one 17 | pub struct CombinedRecordBatchStream { 18 | /// Schema wrapped by Arc 19 | schema: SchemaRef, 20 | /// Stream entries 21 | entries: Vec, 22 | } 23 | 24 | impl CombinedRecordBatchStream { 25 | /// Create an CombinedRecordBatchStream 26 | pub fn new(schema: SchemaRef, entries: Vec) -> Self { 27 | Self { schema, entries } 28 | } 29 | } 30 | 31 | impl RecordBatchStream for CombinedRecordBatchStream { 32 | fn schema(&self) -> SchemaRef { 33 | self.schema.clone() 34 | } 35 | } 36 | 37 | impl Stream for CombinedRecordBatchStream { 38 | type Item = Result; 39 | 40 | fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { 41 | use Poll::*; 42 | 43 | let start = thread_rng_n(self.entries.len() as u32) as usize; 44 | let mut idx = start; 45 | 46 | for _ in 0..self.entries.len() { 47 | let stream = self.entries.get_mut(idx).unwrap(); 48 | 49 | match Pin::new(stream).poll_next(cx) { 50 | Ready(Some(val)) => return Ready(Some(val)), 51 | Ready(None) => { 52 | // Remove the entry 53 | self.entries.swap_remove(idx); 54 | 55 | // Check if this was the last entry, if so the cursor needs 56 | // to wrap 57 | if idx == self.entries.len() { 58 | idx = 0; 59 | } else if idx < start && start <= self.entries.len() { 60 | // The stream being swapped into the current index has 61 | // already been polled, so skip it. 62 | idx = idx.wrapping_add(1) % self.entries.len(); 63 | } 64 | } 65 | Pending => { 66 | idx = idx.wrapping_add(1) % self.entries.len(); 67 | } 68 | } 69 | } 70 | 71 | // If the map is empty, then the stream is complete. 72 | if self.entries.is_empty() { 73 | Ready(None) 74 | } else { 75 | Pending 76 | } 77 | } 78 | } 79 | -------------------------------------------------------------------------------- /src/shuffle/ray_shuffle/reader.rs: -------------------------------------------------------------------------------- 1 | use crate::shuffle::ray_shuffle::CombinedRecordBatchStream; 2 | use datafusion::arrow::datatypes::SchemaRef; 3 | use datafusion::arrow::record_batch::RecordBatch; 4 | use datafusion::common::Statistics; 5 | use datafusion::error::{DataFusionError, Result}; 6 | use datafusion::execution::context::TaskContext; 7 | use datafusion::physical_expr::expressions::UnKnownColumn; 8 | use datafusion::physical_expr::PhysicalSortExpr; 9 | use datafusion::physical_plan::{ 10 | DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, 11 | SendableRecordBatchStream, 12 | }; 13 | use futures::Stream; 14 | use std::any::Any; 15 | use std::collections::HashMap; 16 | use std::fmt::Formatter; 17 | use std::pin::Pin; 18 | use std::sync::{Arc, RwLock}; 19 | use std::task::{Context, Poll}; 20 | 21 | type PartitionId = usize; 22 | type StageId = usize; 23 | 24 | #[derive(Debug)] 25 | pub struct RayShuffleReaderExec { 26 | /// Query stage to read from 27 | pub stage_id: StageId, 28 | /// The output schema of the query stage being read from 29 | schema: SchemaRef, 30 | /// Output partitioning 31 | partitioning: Partitioning, 32 | /// Input streams from Ray object store 33 | input_partitions_map: RwLock>>, // TODO(@lsf) can we not use Rwlock? 34 | } 35 | 36 | impl RayShuffleReaderExec { 37 | pub fn new(stage_id: StageId, schema: SchemaRef, partitioning: Partitioning) -> Self { 38 | let partitioning = match partitioning { 39 | Partitioning::Hash(expr, n) if expr.is_empty() => Partitioning::UnknownPartitioning(n), 40 | Partitioning::Hash(expr, n) => { 41 | // workaround for DataFusion bug https://github.com/apache/arrow-datafusion/issues/5184 42 | Partitioning::Hash( 43 | expr.into_iter() 44 | .filter(|e| e.as_any().downcast_ref::().is_none()) 45 | .collect(), 46 | n, 47 | ) 48 | } 49 | _ => partitioning, 50 | }; 51 | 52 | Self { 53 | stage_id, 54 | schema, 55 | partitioning, 56 | input_partitions_map: RwLock::new(HashMap::new()), 57 | } 58 | } 59 | 60 | pub fn add_input_partition( 61 | &self, 62 | partition: PartitionId, 63 | input_batch: RecordBatch, 64 | ) -> Result<(), DataFusionError> { 65 | let mut map = self.input_partitions_map.write().unwrap(); 66 | let input_partitions = map.entry(partition).or_insert(vec![]); 67 | input_partitions.push(input_batch); 68 | Ok(()) 69 | } 70 | } 71 | 72 | impl ExecutionPlan for RayShuffleReaderExec { 73 | fn as_any(&self) -> &dyn Any { 74 | self 75 | } 76 | 77 | fn schema(&self) -> SchemaRef { 78 | self.schema.clone() 79 | } 80 | 81 | fn output_partitioning(&self) -> Partitioning { 82 | self.partitioning.clone() 83 | } 84 | 85 | fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { 86 | // TODO could be implemented in some cases 87 | None 88 | } 89 | 90 | fn children(&self) -> Vec> { 91 | vec![] 92 | } 93 | 94 | fn with_new_children( 95 | self: Arc, 96 | _: Vec>, 97 | ) -> datafusion::common::Result> { 98 | Ok(self) 99 | } 100 | 101 | fn execute( 102 | &self, 103 | partition: usize, 104 | _context: Arc, 105 | ) -> datafusion::common::Result { 106 | let mut map = self.input_partitions_map.write().expect("got lock"); 107 | let input_objects = map.remove(&partition).unwrap_or(vec![]); 108 | println!( 109 | "RayShuffleReaderExec[stage={}].execute(input_partition={partition}) with {} shuffle inputs", 110 | self.stage_id, 111 | input_objects.len(), 112 | ); 113 | let mut streams = vec![]; 114 | for input in input_objects { 115 | streams.push( 116 | Box::pin(InMemoryShuffleStream::try_new(input)?) as SendableRecordBatchStream 117 | ); 118 | } 119 | Ok(Box::pin(CombinedRecordBatchStream::new( 120 | self.schema.clone(), 121 | streams, 122 | ))) 123 | } 124 | 125 | fn statistics(&self) -> Result { 126 | Ok(Statistics::new_unknown(&self.schema)) 127 | } 128 | } 129 | 130 | impl DisplayAs for RayShuffleReaderExec { 131 | fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { 132 | write!( 133 | f, 134 | "RayShuffleReaderExec(stage_id={}, input_partitioning={:?})", 135 | self.stage_id, self.partitioning 136 | ) 137 | } 138 | } 139 | 140 | struct InMemoryShuffleStream { 141 | batch: Arc, 142 | read: bool, 143 | } 144 | 145 | impl InMemoryShuffleStream { 146 | fn try_new(batch: RecordBatch) -> Result { 147 | Ok(Self { 148 | batch: Arc::new(batch), 149 | read: false, 150 | }) 151 | } 152 | } 153 | 154 | impl Stream for InMemoryShuffleStream { 155 | type Item = datafusion::error::Result; 156 | 157 | fn poll_next(mut self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { 158 | Poll::Ready(if self.read { 159 | None 160 | } else { 161 | self.read = true; 162 | Some(Ok(self.batch.as_ref().clone())) 163 | }) 164 | } 165 | } 166 | 167 | impl RecordBatchStream for InMemoryShuffleStream { 168 | fn schema(&self) -> SchemaRef { 169 | self.batch.schema() 170 | } 171 | } 172 | -------------------------------------------------------------------------------- /src/shuffle/ray_shuffle/writer.rs: -------------------------------------------------------------------------------- 1 | use datafusion::arrow::compute::concat_batches; 2 | use datafusion::arrow::datatypes::SchemaRef; 3 | use datafusion::arrow::record_batch::RecordBatch; 4 | use datafusion::common::{Result, Statistics}; 5 | use datafusion::error::DataFusionError; 6 | use datafusion::execution::context::TaskContext; 7 | use datafusion::physical_expr::expressions::UnKnownColumn; 8 | use datafusion::physical_expr::PhysicalSortExpr; 9 | use datafusion::physical_plan::common::batch_byte_size; 10 | use datafusion::physical_plan::memory::MemoryStream; 11 | use datafusion::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricBuilder}; 12 | use datafusion::physical_plan::repartition::BatchPartitioner; 13 | use datafusion::physical_plan::stream::RecordBatchStreamAdapter; 14 | use datafusion::physical_plan::{ 15 | DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, 16 | }; 17 | use futures::StreamExt; 18 | use futures::TryStreamExt; 19 | use log::debug; 20 | use std::any::Any; 21 | use std::fmt::Formatter; 22 | use std::sync::Arc; 23 | 24 | #[derive(Debug)] 25 | pub struct RayShuffleWriterExec { 26 | pub stage_id: usize, 27 | /// The child execution plan 28 | pub(crate) plan: Arc, 29 | /// Output partitioning 30 | partitioning: Partitioning, 31 | /// Metrics 32 | pub metrics: ExecutionPlanMetricsSet, 33 | } 34 | 35 | impl RayShuffleWriterExec { 36 | pub fn new(stage_id: usize, plan: Arc, partitioning: Partitioning) -> Self { 37 | let partitioning = match partitioning { 38 | Partitioning::Hash(expr, n) if expr.is_empty() => Partitioning::UnknownPartitioning(n), 39 | Partitioning::Hash(expr, n) => { 40 | // workaround for DataFusion bug https://github.com/apache/arrow-datafusion/issues/5184 41 | Partitioning::Hash( 42 | expr.into_iter() 43 | .filter(|e| e.as_any().downcast_ref::().is_none()) 44 | .collect(), 45 | n, 46 | ) 47 | } 48 | _ => partitioning, 49 | }; 50 | 51 | Self { 52 | stage_id, 53 | plan, 54 | partitioning, 55 | metrics: ExecutionPlanMetricsSet::new(), 56 | } 57 | } 58 | } 59 | 60 | impl ExecutionPlan for RayShuffleWriterExec { 61 | fn as_any(&self) -> &dyn Any { 62 | self 63 | } 64 | 65 | fn schema(&self) -> SchemaRef { 66 | self.plan.schema() 67 | } 68 | 69 | fn output_partitioning(&self) -> Partitioning { 70 | self.partitioning.clone() 71 | } 72 | 73 | fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { 74 | // TODO in the case of a single partition of a sorted plan this could be implemented 75 | None 76 | } 77 | 78 | fn children(&self) -> Vec> { 79 | vec![self.plan.clone()] 80 | } 81 | 82 | fn with_new_children( 83 | self: Arc, 84 | _: Vec>, 85 | ) -> Result> { 86 | unimplemented!() 87 | } 88 | 89 | fn execute( 90 | &self, 91 | input_partition: usize, 92 | context: Arc, 93 | ) -> Result { 94 | println!( 95 | "RayShuffleWriterExec[stage={}].execute(input_partition={input_partition})", 96 | self.stage_id 97 | ); 98 | let mut stream = self.plan.execute(input_partition, context)?; 99 | 100 | let stage_id = self.stage_id; 101 | let partitioning = self.output_partitioning(); 102 | let partition_count = partitioning.partition_count(); 103 | let repart_time = 104 | MetricBuilder::new(&self.metrics).subset_time("repart_time", input_partition); 105 | let schema = Arc::new(self.schema().as_ref().clone()); 106 | 107 | let results = async move { 108 | // TODO(@lsf): why can't I reference self in here? 109 | match &partitioning { 110 | Partitioning::UnknownPartitioning(_) => { 111 | let mut writer = InMemoryWriter::new(schema.clone()); 112 | while let Some(result) = stream.next().await { 113 | writer.write(result?)?; 114 | } 115 | println!( 116 | "RayShuffleWriterExec[stage={}] Finished writing shuffle partition 0. Batches: {}. Rows: {}. Bytes: {}.", 117 | stage_id, 118 | writer.num_batches, 119 | writer.num_rows, 120 | writer.num_bytes 121 | ); 122 | MemoryStream::try_new(vec![writer.finish()?], schema, None) 123 | } 124 | Partitioning::Hash(_, _) => { 125 | let mut writers: Vec = vec![]; 126 | for _ in 0..partition_count { 127 | writers.push(InMemoryWriter::new(schema.clone())); 128 | } 129 | 130 | let mut partitioner = 131 | BatchPartitioner::try_new(partitioning, repart_time.clone())?; 132 | 133 | let mut rows = 0; 134 | 135 | while let Some(result) = stream.next().await { 136 | let input_batch = result?; 137 | rows += input_batch.num_rows(); 138 | partitioner.partition(input_batch, |output_partition, output_batch| { 139 | writers[output_partition].write(output_batch) 140 | })?; 141 | } 142 | let mut result_batches = vec![]; 143 | for (i, w) in writers.iter_mut().enumerate() { 144 | println!( 145 | "RayShuffleWriterExec[stage={}] Finished writing shuffle partition {}. Batches: {}. Rows: {}. Bytes: {}.", 146 | stage_id, 147 | i, 148 | w.num_batches, 149 | w.num_rows, 150 | w.num_bytes 151 | ); 152 | result_batches.push(w.finish()?); 153 | } 154 | debug!( 155 | "RayShuffleWriterExec[stage={}] finished processing stream with {rows} rows", 156 | stage_id 157 | ); 158 | MemoryStream::try_new(result_batches, schema, None) 159 | } 160 | _ => unimplemented!(), 161 | } 162 | }; 163 | Ok(Box::pin(RecordBatchStreamAdapter::new( 164 | self.schema(), 165 | futures::stream::once(results).try_flatten(), 166 | ))) 167 | } 168 | 169 | fn statistics(&self) -> Result { 170 | Ok(Statistics::new_unknown(&self.schema())) 171 | } 172 | } 173 | 174 | impl DisplayAs for RayShuffleWriterExec { 175 | fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { 176 | write!( 177 | f, 178 | "RayShuffleWriterExec(stage_id={}, output_partitioning={:?})", 179 | self.stage_id, self.partitioning 180 | ) 181 | } 182 | } 183 | 184 | struct InMemoryWriter { 185 | /// batches buffer 186 | batches: Vec, 187 | /// schema 188 | schema: SchemaRef, 189 | /// batches written 190 | pub num_batches: u64, 191 | /// rows written 192 | pub num_rows: u64, 193 | /// bytes written 194 | pub num_bytes: u64, 195 | } 196 | 197 | impl InMemoryWriter { 198 | fn new(schema: SchemaRef) -> Self { 199 | Self { 200 | batches: vec![], 201 | schema: schema, 202 | num_batches: 0, 203 | num_rows: 0, 204 | num_bytes: 0, 205 | } 206 | } 207 | 208 | fn write(&mut self, batch: RecordBatch) -> Result<()> { 209 | self.num_batches += 1; 210 | self.num_rows += batch.num_rows() as u64; 211 | self.num_bytes += batch_byte_size(&batch) as u64; 212 | self.batches.push(batch); 213 | Ok(()) 214 | } 215 | 216 | fn finish(&self) -> Result { 217 | // TODO(@lsf) Instead of concatenating the batches, return all RecordBatches from 218 | // all partitions in one stream, then return an array of batch offsets. 219 | concat_batches(&self.schema, &self.batches).map_err(DataFusionError::ArrowError) 220 | } 221 | } 222 | -------------------------------------------------------------------------------- /src/shuffle/reader.rs: -------------------------------------------------------------------------------- 1 | use crate::shuffle::ray_shuffle::CombinedRecordBatchStream; 2 | use datafusion::arrow::datatypes::SchemaRef; 3 | use datafusion::arrow::ipc::reader::FileReader; 4 | use datafusion::arrow::record_batch::RecordBatch; 5 | use datafusion::common::Statistics; 6 | use datafusion::error::{DataFusionError, Result}; 7 | use datafusion::execution::context::TaskContext; 8 | use datafusion::physical_expr::expressions::UnKnownColumn; 9 | use datafusion::physical_expr::PhysicalSortExpr; 10 | use datafusion::physical_plan::{ 11 | DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, 12 | SendableRecordBatchStream, 13 | }; 14 | use futures::Stream; 15 | use glob::glob; 16 | use log::debug; 17 | use std::any::Any; 18 | use std::fmt::Formatter; 19 | use std::fs::File; 20 | use std::pin::Pin; 21 | use std::sync::Arc; 22 | use std::task::{Context, Poll}; 23 | 24 | #[derive(Debug)] 25 | pub struct ShuffleReaderExec { 26 | /// Query stage to read from 27 | pub stage_id: usize, 28 | /// The output schema of the query stage being read from 29 | schema: SchemaRef, 30 | /// Output partitioning 31 | partitioning: Partitioning, 32 | /// Directory to read shuffle files from 33 | pub shuffle_dir: String, 34 | } 35 | 36 | impl ShuffleReaderExec { 37 | pub fn new( 38 | stage_id: usize, 39 | schema: SchemaRef, 40 | partitioning: Partitioning, 41 | shuffle_dir: &str, 42 | ) -> Self { 43 | let partitioning = match partitioning { 44 | Partitioning::Hash(expr, n) if expr.is_empty() => Partitioning::UnknownPartitioning(n), 45 | Partitioning::Hash(expr, n) => { 46 | // workaround for DataFusion bug https://github.com/apache/arrow-datafusion/issues/5184 47 | Partitioning::Hash( 48 | expr.into_iter() 49 | .filter(|e| e.as_any().downcast_ref::().is_none()) 50 | .collect(), 51 | n, 52 | ) 53 | } 54 | _ => partitioning, 55 | }; 56 | 57 | Self { 58 | stage_id, 59 | schema, 60 | partitioning, 61 | shuffle_dir: shuffle_dir.to_string(), 62 | } 63 | } 64 | } 65 | 66 | impl ExecutionPlan for ShuffleReaderExec { 67 | fn as_any(&self) -> &dyn Any { 68 | self 69 | } 70 | 71 | fn schema(&self) -> SchemaRef { 72 | self.schema.clone() 73 | } 74 | 75 | fn output_partitioning(&self) -> Partitioning { 76 | self.partitioning.clone() 77 | } 78 | 79 | fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { 80 | // TODO could be implemented in some cases 81 | None 82 | } 83 | 84 | fn children(&self) -> Vec> { 85 | vec![] 86 | } 87 | 88 | fn with_new_children( 89 | self: Arc, 90 | _: Vec>, 91 | ) -> datafusion::common::Result> { 92 | Ok(self) 93 | } 94 | 95 | fn execute( 96 | &self, 97 | partition: usize, 98 | _context: Arc, 99 | ) -> datafusion::common::Result { 100 | let pattern = format!( 101 | "/{}/shuffle_{}_*_{partition}.arrow", 102 | self.shuffle_dir, self.stage_id 103 | ); 104 | let mut streams: Vec = vec![]; 105 | for entry in glob(&pattern).expect("Failed to read glob pattern") { 106 | let file = entry.unwrap(); 107 | debug!( 108 | "ShuffleReaderExec partition {} reading from stage {} file {}", 109 | partition, 110 | self.stage_id, 111 | file.display() 112 | ); 113 | let reader = FileReader::try_new(File::open(&file)?, None)?; 114 | let stream = LocalShuffleStream::new(reader); 115 | if self.schema != stream.schema() { 116 | return Err(DataFusionError::Internal( 117 | "Not all shuffle files have the same schema".to_string(), 118 | )); 119 | } 120 | streams.push(Box::pin(stream)); 121 | } 122 | Ok(Box::pin(CombinedRecordBatchStream::new( 123 | self.schema.clone(), 124 | streams, 125 | ))) 126 | } 127 | 128 | fn statistics(&self) -> Result { 129 | Ok(Statistics::new_unknown(&self.schema)) 130 | } 131 | } 132 | 133 | impl DisplayAs for ShuffleReaderExec { 134 | fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { 135 | write!( 136 | f, 137 | "ShuffleReaderExec(stage_id={}, input_partitioning={:?})", 138 | self.stage_id, self.partitioning 139 | ) 140 | } 141 | } 142 | 143 | struct LocalShuffleStream { 144 | reader: FileReader, 145 | } 146 | 147 | impl LocalShuffleStream { 148 | pub fn new(reader: FileReader) -> Self { 149 | LocalShuffleStream { reader } 150 | } 151 | } 152 | 153 | impl Stream for LocalShuffleStream { 154 | type Item = datafusion::error::Result; 155 | 156 | fn poll_next(mut self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { 157 | if let Some(batch) = self.reader.next() { 158 | return Poll::Ready(Some(batch.map_err(|e| e.into()))); 159 | } 160 | Poll::Ready(None) 161 | } 162 | } 163 | 164 | impl RecordBatchStream for LocalShuffleStream { 165 | fn schema(&self) -> SchemaRef { 166 | self.reader.schema() 167 | } 168 | } 169 | -------------------------------------------------------------------------------- /src/shuffle/writer.rs: -------------------------------------------------------------------------------- 1 | use datafusion::arrow::array::Int32Array; 2 | use datafusion::arrow::datatypes::{DataType, Field, Schema, SchemaRef}; 3 | use datafusion::arrow::ipc::writer::FileWriter; 4 | use datafusion::arrow::record_batch::RecordBatch; 5 | use datafusion::arrow::util::pretty::pretty_format_batches; 6 | use datafusion::common::{Result, Statistics}; 7 | use datafusion::execution::context::TaskContext; 8 | use datafusion::physical_expr::expressions::UnKnownColumn; 9 | use datafusion::physical_expr::PhysicalSortExpr; 10 | use datafusion::physical_plan::common::{batch_byte_size, IPCWriter}; 11 | use datafusion::physical_plan::memory::MemoryStream; 12 | use datafusion::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricBuilder}; 13 | use datafusion::physical_plan::repartition::BatchPartitioner; 14 | use datafusion::physical_plan::stream::RecordBatchStreamAdapter; 15 | use datafusion::physical_plan::{ 16 | metrics, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, 17 | SendableRecordBatchStream, 18 | }; 19 | use datafusion_proto::protobuf::PartitionStats; 20 | use futures::StreamExt; 21 | use futures::TryStreamExt; 22 | use log::debug; 23 | use std::any::Any; 24 | use std::fmt::Formatter; 25 | use std::fs::File; 26 | use std::path::Path; 27 | use std::pin::Pin; 28 | use std::sync::Arc; 29 | 30 | #[derive(Debug)] 31 | pub struct ShuffleWriterExec { 32 | pub stage_id: usize, 33 | pub(crate) plan: Arc, 34 | /// Output partitioning 35 | partitioning: Partitioning, 36 | /// Directory to write shuffle files from 37 | pub shuffle_dir: String, 38 | /// Metrics 39 | pub metrics: ExecutionPlanMetricsSet, 40 | } 41 | 42 | impl ShuffleWriterExec { 43 | pub fn new( 44 | stage_id: usize, 45 | plan: Arc, 46 | partitioning: Partitioning, 47 | shuffle_dir: &str, 48 | ) -> Self { 49 | let partitioning = match partitioning { 50 | Partitioning::Hash(expr, n) if expr.is_empty() => Partitioning::UnknownPartitioning(n), 51 | Partitioning::Hash(expr, n) => { 52 | // workaround for DataFusion bug https://github.com/apache/arrow-datafusion/issues/5184 53 | Partitioning::Hash( 54 | expr.into_iter() 55 | .filter(|e| e.as_any().downcast_ref::().is_none()) 56 | .collect(), 57 | n, 58 | ) 59 | } 60 | _ => partitioning, 61 | }; 62 | 63 | Self { 64 | stage_id, 65 | plan, 66 | partitioning, 67 | shuffle_dir: shuffle_dir.to_string(), 68 | metrics: ExecutionPlanMetricsSet::new(), 69 | } 70 | } 71 | } 72 | 73 | impl ExecutionPlan for ShuffleWriterExec { 74 | fn as_any(&self) -> &dyn Any { 75 | self 76 | } 77 | 78 | fn schema(&self) -> SchemaRef { 79 | self.plan.schema() 80 | } 81 | 82 | fn output_partitioning(&self) -> Partitioning { 83 | self.partitioning.clone() 84 | } 85 | 86 | fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { 87 | // TODO in the case of a single partition of a sorted plan this could be implemented 88 | None 89 | } 90 | 91 | fn children(&self) -> Vec> { 92 | vec![self.plan.clone()] 93 | } 94 | 95 | fn with_new_children( 96 | self: Arc, 97 | _: Vec>, 98 | ) -> Result> { 99 | unimplemented!() 100 | } 101 | 102 | fn execute( 103 | &self, 104 | input_partition: usize, 105 | context: Arc, 106 | ) -> Result { 107 | debug!( 108 | "ShuffleWriterExec[stage={}].execute(input_partition={input_partition})", 109 | self.stage_id 110 | ); 111 | 112 | let mut stream = self.plan.execute(input_partition, context)?; 113 | let write_time = 114 | MetricBuilder::new(&self.metrics).subset_time("write_time", input_partition); 115 | let repart_time = 116 | MetricBuilder::new(&self.metrics).subset_time("repart_time", input_partition); 117 | 118 | let stage_id = self.stage_id; 119 | let partitioning = self.output_partitioning(); 120 | let partition_count = partitioning.partition_count(); 121 | let shuffle_dir = self.shuffle_dir.clone(); 122 | 123 | let results = async move { 124 | match &partitioning { 125 | Partitioning::RoundRobinBatch(_) => { 126 | unimplemented!() 127 | } 128 | Partitioning::UnknownPartitioning(_) => { 129 | // stream the results from the query, preserving the input partitioning 130 | let file = 131 | format!("/{shuffle_dir}/shuffle_{stage_id}_{input_partition}_0.arrow"); 132 | debug!("Executing query and writing results to {file}"); 133 | let stats = write_stream_to_disk(&mut stream, &file, &write_time).await?; 134 | debug!( 135 | "Query completed. Shuffle write time: {}. Rows: {}.", 136 | write_time, stats.num_rows 137 | ); 138 | } 139 | Partitioning::Hash(_, _) => { 140 | // we won't necessary produce output for every possible partition, so we 141 | // create writers on demand 142 | let mut writers: Vec> = vec![]; 143 | for _ in 0..partition_count { 144 | writers.push(None); 145 | } 146 | 147 | let mut partitioner = 148 | BatchPartitioner::try_new(partitioning, repart_time.clone())?; 149 | 150 | let mut rows = 0; 151 | 152 | while let Some(result) = stream.next().await { 153 | let input_batch = result?; 154 | rows += input_batch.num_rows(); 155 | 156 | debug!( 157 | "ShuffleWriterExec[stage={}] writing batch:\n{}", 158 | stage_id, 159 | pretty_format_batches(&[input_batch.clone()])? 160 | ); 161 | 162 | //write_metrics.input_rows.add(input_batch.num_rows()); 163 | 164 | partitioner.partition(input_batch, |output_partition, output_batch| { 165 | match &mut writers[output_partition] { 166 | Some(w) => { 167 | w.write(&output_batch)?; 168 | } 169 | None => { 170 | let path = format!( 171 | "/{shuffle_dir}/shuffle_{stage_id}_{input_partition}_{output_partition}.arrow", 172 | ); 173 | let path = Path::new(&path); 174 | debug!("ShuffleWriterExec[stage={}] Writing results to {:?}", stage_id, path); 175 | 176 | let mut writer = IPCWriter::new(path, stream.schema().as_ref())?; 177 | 178 | writer.write(&output_batch)?; 179 | writers[output_partition] = Some(writer); 180 | } 181 | } 182 | Ok(()) 183 | })?; 184 | } 185 | 186 | for (i, w) in writers.iter_mut().enumerate() { 187 | match w { 188 | Some(w) => { 189 | w.finish()?; 190 | debug!( 191 | "ShuffleWriterExec[stage={}] Finished writing shuffle partition {} at {:?}. Batches: {}. Rows: {}. Bytes: {}.", 192 | stage_id, 193 | i, 194 | w.path(), 195 | w.num_batches, 196 | w.num_rows, 197 | w.num_bytes 198 | ); 199 | } 200 | None => {} 201 | } 202 | } 203 | debug!( 204 | "ShuffleWriterExec[stage={}] Finished processing stream with {rows} rows", 205 | stage_id 206 | ); 207 | } 208 | } 209 | 210 | // create a dummy batch to return - later this could be metadata about the 211 | // shuffle partitions that were written out 212 | let schema = Arc::new(Schema::new(vec![ 213 | Field::new("shuffle_repart_time", DataType::Int32, true), 214 | Field::new("shuffle_write_time", DataType::Int32, true), 215 | ])); 216 | let arr_repart_time = Int32Array::from(vec![repart_time.value() as i32]); 217 | let arr_write_time = Int32Array::from(vec![write_time.value() as i32]); 218 | let batch = RecordBatch::try_new( 219 | schema.clone(), 220 | vec![Arc::new(arr_repart_time), Arc::new(arr_write_time)], 221 | )?; 222 | 223 | // return as a stream 224 | MemoryStream::try_new(vec![batch], schema, None) 225 | }; 226 | let schema = self.schema(); 227 | Ok(Box::pin(RecordBatchStreamAdapter::new( 228 | schema, 229 | futures::stream::once(results).try_flatten(), 230 | ))) 231 | } 232 | 233 | fn statistics(&self) -> Result { 234 | Ok(Statistics::new_unknown(&self.schema())) 235 | } 236 | } 237 | 238 | impl DisplayAs for ShuffleWriterExec { 239 | fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { 240 | write!( 241 | f, 242 | "ShuffleWriterExec(stage_id={}, output_partitioning={:?})", 243 | self.stage_id, self.partitioning 244 | ) 245 | } 246 | } 247 | 248 | /// Stream data to disk in Arrow IPC format 249 | pub async fn write_stream_to_disk( 250 | stream: &mut Pin>, 251 | path: &str, 252 | disk_write_metric: &metrics::Time, 253 | ) -> Result { 254 | let file = File::create(path).unwrap(); 255 | 256 | /*.map_err(|e| { 257 | error!("Failed to create partition file at {}: {:?}", path, e); 258 | BallistaError::IoError(e) 259 | })?;*/ 260 | 261 | let mut num_rows = 0; 262 | let mut num_batches = 0; 263 | let mut num_bytes = 0; 264 | let mut writer = FileWriter::try_new(file, stream.schema().as_ref())?; 265 | 266 | while let Some(result) = stream.next().await { 267 | let batch = result?; 268 | 269 | let batch_size_bytes: usize = batch_byte_size(&batch); 270 | num_batches += 1; 271 | num_rows += batch.num_rows(); 272 | num_bytes += batch_size_bytes; 273 | 274 | let timer = disk_write_metric.timer(); 275 | writer.write(&batch)?; 276 | timer.done(); 277 | } 278 | let timer = disk_write_metric.timer(); 279 | writer.finish()?; 280 | timer.done(); 281 | Ok(PartitionStats { 282 | num_rows: num_rows as i64, 283 | num_batches: num_batches as i64, 284 | num_bytes: num_bytes as i64, 285 | column_stats: vec![], 286 | }) 287 | } 288 | -------------------------------------------------------------------------------- /src/utils.rs: -------------------------------------------------------------------------------- 1 | use pyo3::Python; 2 | use std::future::Future; 3 | use tokio::runtime::Runtime; 4 | 5 | /// Utility to collect rust futures with GIL released 6 | pub(crate) fn wait_for_future(py: Python, f: F) -> F::Output 7 | where 8 | F: Send, 9 | F::Output: Send, 10 | { 11 | let rt = Runtime::new().unwrap(); 12 | py.allow_threads(|| rt.block_on(f)) 13 | } 14 | -------------------------------------------------------------------------------- /testdata/expected-plans/q1.txt: -------------------------------------------------------------------------------- 1 | DataFusion Logical Plan 2 | ======================= 3 | 4 | Sort: lineitem.l_returnflag ASC NULLS LAST, lineitem.l_linestatus ASC NULLS LAST 5 | 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 6 | 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))]] 7 | 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 8 | Filter: lineitem.l_shipdate <= Date32("10493") 9 | TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], partial_filters=[lineitem.l_shipdate <= Date32("10493")] 10 | 11 | DataFusion Physical Plan 12 | ======================== 13 | 14 | SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] 15 | SortExec: expr=[l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] 16 | 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] 17 | 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))] 18 | CoalesceBatchesExec: target_batch_size=8192 19 | RepartitionExec: partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 4), input_partitions=4 20 | 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))] 21 | 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] 22 | CoalesceBatchesExec: target_batch_size=8192 23 | FilterExec: l_shipdate@6 <= 10493 24 | 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] 25 | 26 | RaySQL Plan 27 | =========== 28 | 29 | Query Stage #0 (4 -> 4): 30 | ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 4)) 31 | 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))] 32 | 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] 33 | CoalesceBatchesExec: target_batch_size=8192 34 | FilterExec: l_shipdate@6 <= 10493 35 | 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] 36 | 37 | Query Stage #1 (4 -> 4): 38 | ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 4)) 39 | SortExec: expr=[l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] 40 | 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] 41 | 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))] 42 | CoalesceBatchesExec: target_batch_size=8192 43 | ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 4)) 44 | 45 | Query Stage #2 (4 -> 1): 46 | SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] 47 | ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 4)) 48 | 49 | -------------------------------------------------------------------------------- /testdata/expected-plans/q12.txt: -------------------------------------------------------------------------------- 1 | DataFusion Logical Plan 2 | ======================= 3 | 4 | Sort: lineitem.l_shipmode ASC NULLS LAST 5 | 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 6 | 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)]] 7 | Projection: orders.o_orderpriority, lineitem.l_shipmode 8 | Inner Join: orders.o_orderkey = lineitem.l_orderkey 9 | TableScan: orders projection=[o_orderkey, o_orderpriority] 10 | Projection: lineitem.l_orderkey, lineitem.l_shipmode 11 | 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") 12 | 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")] 13 | 14 | DataFusion Physical Plan 15 | ======================== 16 | 17 | SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] 18 | SortExec: expr=[l_shipmode@0 ASC NULLS LAST] 19 | 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] 20 | 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)] 21 | CoalesceBatchesExec: target_batch_size=8192 22 | RepartitionExec: partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 4), input_partitions=4 23 | 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)] 24 | ProjectionExec: expr=[o_orderpriority@1 as o_orderpriority, l_shipmode@3 as l_shipmode] 25 | CoalesceBatchesExec: target_batch_size=8192 26 | HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 0 }, Column { name: "l_orderkey", index: 0 })] 27 | CoalesceBatchesExec: target_batch_size=8192 28 | RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4), input_partitions=4 29 | 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] 30 | CoalesceBatchesExec: target_batch_size=8192 31 | RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4 32 | ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] 33 | CoalesceBatchesExec: target_batch_size=8192 34 | 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 35 | 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] 36 | 37 | RaySQL Plan 38 | =========== 39 | 40 | Query Stage #0 (4 -> 4): 41 | ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) 42 | 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] 43 | 44 | Query Stage #1 (4 -> 4): 45 | ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) 46 | ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] 47 | CoalesceBatchesExec: target_batch_size=8192 48 | 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 49 | 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] 50 | 51 | Query Stage #2 (4 -> 4): 52 | ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 4)) 53 | 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)] 54 | ProjectionExec: expr=[o_orderpriority@1 as o_orderpriority, l_shipmode@3 as l_shipmode] 55 | CoalesceBatchesExec: target_batch_size=8192 56 | HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 0 }, Column { name: "l_orderkey", index: 0 })] 57 | CoalesceBatchesExec: target_batch_size=8192 58 | ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) 59 | CoalesceBatchesExec: target_batch_size=8192 60 | ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) 61 | 62 | Query Stage #3 (4 -> 4): 63 | ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 4)) 64 | SortExec: expr=[l_shipmode@0 ASC NULLS LAST] 65 | 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] 66 | 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)] 67 | CoalesceBatchesExec: target_batch_size=8192 68 | ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 4)) 69 | 70 | Query Stage #4 (4 -> 1): 71 | SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] 72 | ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 4)) 73 | 74 | -------------------------------------------------------------------------------- /testdata/expected-plans/q13.txt: -------------------------------------------------------------------------------- 1 | DataFusion Logical Plan 2 | ======================= 3 | 4 | Sort: custdist DESC NULLS FIRST, c_count DESC NULLS FIRST 5 | Projection: c_count, COUNT(UInt8(1)) AS custdist 6 | Aggregate: groupBy=[[c_count]], aggr=[[COUNT(UInt8(1))]] 7 | Projection: c_orders.COUNT(orders.o_orderkey) AS c_count 8 | SubqueryAlias: c_orders 9 | Projection: COUNT(orders.o_orderkey) 10 | Aggregate: groupBy=[[customer.c_custkey]], aggr=[[COUNT(orders.o_orderkey)]] 11 | Projection: customer.c_custkey, orders.o_orderkey 12 | Left Join: customer.c_custkey = orders.o_custkey 13 | TableScan: customer projection=[c_custkey] 14 | Projection: orders.o_orderkey, orders.o_custkey 15 | Filter: orders.o_comment NOT LIKE Utf8("%express%requests%") 16 | TableScan: orders projection=[o_orderkey, o_custkey, o_comment], partial_filters=[orders.o_comment NOT LIKE Utf8("%express%requests%")] 17 | 18 | DataFusion Physical Plan 19 | ======================== 20 | 21 | SortPreservingMergeExec: [custdist@1 DESC,c_count@0 DESC] 22 | SortExec: expr=[custdist@1 DESC,c_count@0 DESC] 23 | ProjectionExec: expr=[c_count@0 as c_count, COUNT(UInt8(1))@1 as custdist] 24 | AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[COUNT(UInt8(1))] 25 | CoalesceBatchesExec: target_batch_size=8192 26 | RepartitionExec: partitioning=Hash([Column { name: "c_count", index: 0 }], 4), input_partitions=4 27 | AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[COUNT(UInt8(1))] 28 | ProjectionExec: expr=[COUNT(orders.o_orderkey)@0 as c_count] 29 | ProjectionExec: expr=[COUNT(orders.o_orderkey)@1 as COUNT(orders.o_orderkey)] 30 | AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)] 31 | AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)] 32 | ProjectionExec: expr=[c_custkey@0 as c_custkey, o_orderkey@1 as o_orderkey] 33 | CoalesceBatchesExec: target_batch_size=8192 34 | HashJoinExec: mode=Partitioned, join_type=Left, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })] 35 | CoalesceBatchesExec: target_batch_size=8192 36 | RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4), input_partitions=4 37 | 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] 38 | CoalesceBatchesExec: target_batch_size=8192 39 | RepartitionExec: partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4), input_partitions=4 40 | ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] 41 | CoalesceBatchesExec: target_batch_size=8192 42 | FilterExec: o_comment@2 NOT LIKE %express%requests% 43 | 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] 44 | 45 | RaySQL Plan 46 | =========== 47 | 48 | Query Stage #0 (4 -> 4): 49 | ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) 50 | 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] 51 | 52 | Query Stage #1 (4 -> 4): 53 | ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) 54 | ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] 55 | CoalesceBatchesExec: target_batch_size=8192 56 | FilterExec: o_comment@2 NOT LIKE %express%requests% 57 | 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] 58 | 59 | Query Stage #2 (4 -> 4): 60 | ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "c_count", index: 0 }], 4)) 61 | AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[COUNT(UInt8(1))] 62 | ProjectionExec: expr=[COUNT(orders.o_orderkey)@0 as c_count] 63 | ProjectionExec: expr=[COUNT(orders.o_orderkey)@1 as COUNT(orders.o_orderkey)] 64 | AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)] 65 | AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)] 66 | ProjectionExec: expr=[c_custkey@0 as c_custkey, o_orderkey@1 as o_orderkey] 67 | CoalesceBatchesExec: target_batch_size=8192 68 | HashJoinExec: mode=Partitioned, join_type=Left, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })] 69 | CoalesceBatchesExec: target_batch_size=8192 70 | ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) 71 | CoalesceBatchesExec: target_batch_size=8192 72 | ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) 73 | 74 | Query Stage #3 (4 -> 4): 75 | ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "c_count", index: 0 }], 4)) 76 | SortExec: expr=[custdist@1 DESC,c_count@0 DESC] 77 | ProjectionExec: expr=[c_count@0 as c_count, COUNT(UInt8(1))@1 as custdist] 78 | AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[COUNT(UInt8(1))] 79 | CoalesceBatchesExec: target_batch_size=8192 80 | ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "c_count", index: 0 }], 4)) 81 | 82 | Query Stage #4 (4 -> 1): 83 | SortPreservingMergeExec: [custdist@1 DESC,c_count@0 DESC] 84 | ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "c_count", index: 0 }], 4)) 85 | 86 | -------------------------------------------------------------------------------- /testdata/expected-plans/q14.txt: -------------------------------------------------------------------------------- 1 | DataFusion Logical Plan 2 | ======================= 3 | 4 | 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 5 | 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)]] 6 | 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 7 | Inner Join: lineitem.l_partkey = part.p_partkey 8 | Projection: lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount 9 | Filter: lineitem.l_shipdate >= Date32("9162") AND lineitem.l_shipdate < Date32("9190") 10 | TableScan: lineitem projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9162"), lineitem.l_shipdate < Date32("9190")] 11 | TableScan: part projection=[p_partkey, p_type] 12 | 13 | DataFusion Physical Plan 14 | ======================== 15 | 16 | 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] 17 | 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)] 18 | CoalescePartitionsExec 19 | 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)] 20 | 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] 21 | CoalesceBatchesExec: target_batch_size=8192 22 | HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })] 23 | CoalesceBatchesExec: target_batch_size=8192 24 | RepartitionExec: partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4), input_partitions=4 25 | ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] 26 | CoalesceBatchesExec: target_batch_size=8192 27 | FilterExec: l_shipdate@3 >= 9162 AND l_shipdate@3 < 9190 28 | 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] 29 | CoalesceBatchesExec: target_batch_size=8192 30 | RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4), input_partitions=4 31 | 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] 32 | 33 | RaySQL Plan 34 | =========== 35 | 36 | Query Stage #0 (4 -> 4): 37 | ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4)) 38 | ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] 39 | CoalesceBatchesExec: target_batch_size=8192 40 | FilterExec: l_shipdate@3 >= 9162 AND l_shipdate@3 < 9190 41 | 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] 42 | 43 | Query Stage #1 (4 -> 4): 44 | ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) 45 | 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] 46 | 47 | Query Stage #2 (4 -> 4): 48 | ShuffleWriterExec(stage_id=2, output_partitioning=Hash([], 4)) 49 | 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)] 50 | 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] 51 | CoalesceBatchesExec: target_batch_size=8192 52 | HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })] 53 | CoalesceBatchesExec: target_batch_size=8192 54 | ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4)) 55 | CoalesceBatchesExec: target_batch_size=8192 56 | ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) 57 | 58 | Query Stage #3 (1 -> 1): 59 | 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] 60 | 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)] 61 | CoalescePartitionsExec 62 | ShuffleReaderExec(stage_id=2, input_partitioning=Hash([], 4)) 63 | 64 | -------------------------------------------------------------------------------- /testdata/expected-plans/q4.txt: -------------------------------------------------------------------------------- 1 | DataFusion Logical Plan 2 | ======================= 3 | 4 | Sort: orders.o_orderpriority ASC NULLS LAST 5 | Projection: orders.o_orderpriority, COUNT(UInt8(1)) AS order_count 6 | Aggregate: groupBy=[[orders.o_orderpriority]], aggr=[[COUNT(UInt8(1))]] 7 | Projection: orders.o_orderpriority 8 | LeftSemi Join: orders.o_orderkey = lineitem.l_orderkey 9 | Projection: orders.o_orderkey, orders.o_orderpriority 10 | Filter: orders.o_orderdate >= Date32("9221") AND orders.o_orderdate < Date32("9312") 11 | TableScan: orders projection=[o_orderkey, o_orderdate, o_orderpriority], partial_filters=[orders.o_orderdate >= Date32("9221"), orders.o_orderdate < Date32("9312")] 12 | Projection: lineitem.l_orderkey 13 | Filter: lineitem.l_commitdate < lineitem.l_receiptdate 14 | TableScan: lineitem projection=[l_orderkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_commitdate < lineitem.l_receiptdate] 15 | 16 | DataFusion Physical Plan 17 | ======================== 18 | 19 | SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] 20 | SortExec: expr=[o_orderpriority@0 ASC NULLS LAST] 21 | ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, COUNT(UInt8(1))@1 as order_count] 22 | AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(UInt8(1))] 23 | CoalesceBatchesExec: target_batch_size=8192 24 | RepartitionExec: partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 4), input_partitions=4 25 | AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(UInt8(1))] 26 | ProjectionExec: expr=[o_orderpriority@1 as o_orderpriority] 27 | CoalesceBatchesExec: target_batch_size=8192 28 | HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "o_orderkey", index: 0 }, Column { name: "l_orderkey", index: 0 })] 29 | CoalesceBatchesExec: target_batch_size=8192 30 | RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4), input_partitions=4 31 | ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] 32 | CoalesceBatchesExec: target_batch_size=8192 33 | FilterExec: o_orderdate@1 >= 9221 AND o_orderdate@1 < 9312 34 | 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] 35 | CoalesceBatchesExec: target_batch_size=8192 36 | RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4 37 | ProjectionExec: expr=[l_orderkey@0 as l_orderkey] 38 | CoalesceBatchesExec: target_batch_size=8192 39 | FilterExec: l_commitdate@1 < l_receiptdate@2 40 | 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] 41 | 42 | RaySQL Plan 43 | =========== 44 | 45 | Query Stage #0 (4 -> 4): 46 | ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) 47 | ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] 48 | CoalesceBatchesExec: target_batch_size=8192 49 | FilterExec: o_orderdate@1 >= 9221 AND o_orderdate@1 < 9312 50 | 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] 51 | 52 | Query Stage #1 (4 -> 4): 53 | ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) 54 | ProjectionExec: expr=[l_orderkey@0 as l_orderkey] 55 | CoalesceBatchesExec: target_batch_size=8192 56 | FilterExec: l_commitdate@1 < l_receiptdate@2 57 | 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] 58 | 59 | Query Stage #2 (4 -> 4): 60 | ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 4)) 61 | AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(UInt8(1))] 62 | ProjectionExec: expr=[o_orderpriority@1 as o_orderpriority] 63 | CoalesceBatchesExec: target_batch_size=8192 64 | HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "o_orderkey", index: 0 }, Column { name: "l_orderkey", index: 0 })] 65 | CoalesceBatchesExec: target_batch_size=8192 66 | ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) 67 | CoalesceBatchesExec: target_batch_size=8192 68 | ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) 69 | 70 | Query Stage #3 (4 -> 4): 71 | ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 4)) 72 | SortExec: expr=[o_orderpriority@0 ASC NULLS LAST] 73 | ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, COUNT(UInt8(1))@1 as order_count] 74 | AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(UInt8(1))] 75 | CoalesceBatchesExec: target_batch_size=8192 76 | ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 4)) 77 | 78 | Query Stage #4 (4 -> 1): 79 | SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] 80 | ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 4)) 81 | 82 | -------------------------------------------------------------------------------- /testdata/expected-plans/q6.txt: -------------------------------------------------------------------------------- 1 | DataFusion Logical Plan 2 | ======================= 3 | 4 | Projection: SUM(lineitem.l_extendedprice * lineitem.l_discount) AS revenue 5 | Aggregate: groupBy=[[]], aggr=[[SUM(lineitem.l_extendedprice * lineitem.l_discount)]] 6 | Projection: lineitem.l_extendedprice, lineitem.l_discount 7 | 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) 8 | 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)] 9 | 10 | DataFusion Physical Plan 11 | ======================== 12 | 13 | ProjectionExec: expr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue] 14 | AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] 15 | CoalescePartitionsExec 16 | AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] 17 | ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] 18 | CoalesceBatchesExec: target_batch_size=8192 19 | 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 20 | 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] 21 | 22 | RaySQL Plan 23 | =========== 24 | 25 | Query Stage #0 (4 -> 1): 26 | ShuffleWriterExec(stage_id=0, output_partitioning=UnknownPartitioning(4)) 27 | AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] 28 | ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] 29 | CoalesceBatchesExec: target_batch_size=8192 30 | 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 31 | 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] 32 | 33 | Query Stage #1 (1 -> 1): 34 | ProjectionExec: expr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue] 35 | AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] 36 | CoalescePartitionsExec 37 | ShuffleReaderExec(stage_id=0, input_partitioning=UnknownPartitioning(4)) 38 | 39 | -------------------------------------------------------------------------------- /testdata/queries/q1.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 1 derived from TPC-H query 1 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | l_returnflag, 5 | l_linestatus, 6 | sum(l_quantity) as sum_qty, 7 | sum(l_extendedprice) as sum_base_price, 8 | sum(l_extendedprice * (1 - l_discount)) as sum_disc_price, 9 | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge, 10 | avg(l_quantity) as avg_qty, 11 | avg(l_extendedprice) as avg_price, 12 | avg(l_discount) as avg_disc, 13 | count(*) as count_order 14 | from 15 | lineitem 16 | where 17 | l_shipdate <= date '1998-12-01' - interval '68 days' 18 | group by 19 | l_returnflag, 20 | l_linestatus 21 | order by 22 | l_returnflag, 23 | l_linestatus; 24 | -------------------------------------------------------------------------------- /testdata/queries/q10.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 10 derived from TPC-H query 10 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | c_custkey, 5 | c_name, 6 | sum(l_extendedprice * (1 - l_discount)) as revenue, 7 | c_acctbal, 8 | n_name, 9 | c_address, 10 | c_phone, 11 | c_comment 12 | from 13 | customer, 14 | orders, 15 | lineitem, 16 | nation 17 | where 18 | c_custkey = o_custkey 19 | and l_orderkey = o_orderkey 20 | and o_orderdate >= date '1993-07-01' 21 | and o_orderdate < date '1993-07-01' + interval '3' month 22 | and l_returnflag = 'R' 23 | and c_nationkey = n_nationkey 24 | group by 25 | c_custkey, 26 | c_name, 27 | c_acctbal, 28 | c_phone, 29 | n_name, 30 | c_address, 31 | c_comment 32 | order by 33 | revenue desc limit 20; 34 | -------------------------------------------------------------------------------- /testdata/queries/q11.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 11 derived from TPC-H query 11 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | ps_partkey, 5 | sum(ps_supplycost * ps_availqty) as value 6 | from 7 | partsupp, 8 | supplier, 9 | nation 10 | where 11 | ps_suppkey = s_suppkey 12 | and s_nationkey = n_nationkey 13 | and n_name = 'ALGERIA' 14 | group by 15 | ps_partkey having 16 | sum(ps_supplycost * ps_availqty) > ( 17 | select 18 | sum(ps_supplycost * ps_availqty) * 0.0001000000 19 | from 20 | partsupp, 21 | supplier, 22 | nation 23 | where 24 | ps_suppkey = s_suppkey 25 | and s_nationkey = n_nationkey 26 | and n_name = 'ALGERIA' 27 | ) 28 | order by 29 | value desc; 30 | -------------------------------------------------------------------------------- /testdata/queries/q12.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 12 derived from TPC-H query 12 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | l_shipmode, 5 | sum(case 6 | when o_orderpriority = '1-URGENT' 7 | or o_orderpriority = '2-HIGH' 8 | then 1 9 | else 0 10 | end) as high_line_count, 11 | sum(case 12 | when o_orderpriority <> '1-URGENT' 13 | and o_orderpriority <> '2-HIGH' 14 | then 1 15 | else 0 16 | end) as low_line_count 17 | from 18 | orders, 19 | lineitem 20 | where 21 | o_orderkey = l_orderkey 22 | and l_shipmode in ('FOB', 'SHIP') 23 | and l_commitdate < l_receiptdate 24 | and l_shipdate < l_commitdate 25 | and l_receiptdate >= date '1995-01-01' 26 | and l_receiptdate < date '1995-01-01' + interval '1' year 27 | group by 28 | l_shipmode 29 | order by 30 | l_shipmode; 31 | -------------------------------------------------------------------------------- /testdata/queries/q13.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 13 derived from TPC-H query 13 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | c_count, 5 | count(*) as custdist 6 | from 7 | ( 8 | select 9 | c_custkey, 10 | count(o_orderkey) 11 | from 12 | customer left outer join orders on 13 | c_custkey = o_custkey 14 | and o_comment not like '%express%requests%' 15 | group by 16 | c_custkey 17 | ) as c_orders (c_custkey, c_count) 18 | group by 19 | c_count 20 | order by 21 | custdist desc, 22 | c_count desc; 23 | -------------------------------------------------------------------------------- /testdata/queries/q14.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 14 derived from TPC-H query 14 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | 100.00 * sum(case 5 | when p_type like 'PROMO%' 6 | then l_extendedprice * (1 - l_discount) 7 | else 0 8 | end) / sum(l_extendedprice * (1 - l_discount)) as promo_revenue 9 | from 10 | lineitem, 11 | part 12 | where 13 | l_partkey = p_partkey 14 | and l_shipdate >= date '1995-02-01' 15 | and l_shipdate < date '1995-02-01' + interval '1' month; 16 | -------------------------------------------------------------------------------- /testdata/queries/q15.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 15 derived from TPC-H query 15 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | create view revenue0 (supplier_no, total_revenue) as 4 | select 5 | l_suppkey, 6 | sum(l_extendedprice * (1 - l_discount)) 7 | from 8 | lineitem 9 | where 10 | l_shipdate >= date '1996-08-01' 11 | and l_shipdate < date '1996-08-01' + interval '3' month 12 | group by 13 | l_suppkey; 14 | select 15 | s_suppkey, 16 | s_name, 17 | s_address, 18 | s_phone, 19 | total_revenue 20 | from 21 | supplier, 22 | revenue0 23 | where 24 | s_suppkey = supplier_no 25 | and total_revenue = ( 26 | select 27 | max(total_revenue) 28 | from 29 | revenue0 30 | ) 31 | order by 32 | s_suppkey; 33 | drop view revenue0; 34 | -------------------------------------------------------------------------------- /testdata/queries/q16.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 16 derived from TPC-H query 16 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | p_brand, 5 | p_type, 6 | p_size, 7 | count(distinct ps_suppkey) as supplier_cnt 8 | from 9 | partsupp, 10 | part 11 | where 12 | p_partkey = ps_partkey 13 | and p_brand <> 'Brand#14' 14 | and p_type not like 'SMALL PLATED%' 15 | and p_size in (14, 6, 5, 31, 49, 15, 41, 47) 16 | and ps_suppkey not in ( 17 | select 18 | s_suppkey 19 | from 20 | supplier 21 | where 22 | s_comment like '%Customer%Complaints%' 23 | ) 24 | group by 25 | p_brand, 26 | p_type, 27 | p_size 28 | order by 29 | supplier_cnt desc, 30 | p_brand, 31 | p_type, 32 | p_size; 33 | -------------------------------------------------------------------------------- /testdata/queries/q17.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 17 derived from TPC-H query 17 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | sum(l_extendedprice) / 7.0 as avg_yearly 5 | from 6 | lineitem, 7 | part 8 | where 9 | p_partkey = l_partkey 10 | and p_brand = 'Brand#42' 11 | and p_container = 'LG BAG' 12 | and l_quantity < ( 13 | select 14 | 0.2 * avg(l_quantity) 15 | from 16 | lineitem 17 | where 18 | l_partkey = p_partkey 19 | ); 20 | -------------------------------------------------------------------------------- /testdata/queries/q18.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 18 derived from TPC-H query 18 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | c_name, 5 | c_custkey, 6 | o_orderkey, 7 | o_orderdate, 8 | o_totalprice, 9 | sum(l_quantity) 10 | from 11 | customer, 12 | orders, 13 | lineitem 14 | where 15 | o_orderkey in ( 16 | select 17 | l_orderkey 18 | from 19 | lineitem 20 | group by 21 | l_orderkey having 22 | sum(l_quantity) > 313 23 | ) 24 | and c_custkey = o_custkey 25 | and o_orderkey = l_orderkey 26 | group by 27 | c_name, 28 | c_custkey, 29 | o_orderkey, 30 | o_orderdate, 31 | o_totalprice 32 | order by 33 | o_totalprice desc, 34 | o_orderdate limit 100; 35 | -------------------------------------------------------------------------------- /testdata/queries/q19.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 19 derived from TPC-H query 19 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | sum(l_extendedprice* (1 - l_discount)) as revenue 5 | from 6 | lineitem, 7 | part 8 | where 9 | ( 10 | p_partkey = l_partkey 11 | and p_brand = 'Brand#21' 12 | and p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG') 13 | and l_quantity >= 8 and l_quantity <= 8 + 10 14 | and p_size between 1 and 5 15 | and l_shipmode in ('AIR', 'AIR REG') 16 | and l_shipinstruct = 'DELIVER IN PERSON' 17 | ) 18 | or 19 | ( 20 | p_partkey = l_partkey 21 | and p_brand = 'Brand#13' 22 | and p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK') 23 | and l_quantity >= 20 and l_quantity <= 20 + 10 24 | and p_size between 1 and 10 25 | and l_shipmode in ('AIR', 'AIR REG') 26 | and l_shipinstruct = 'DELIVER IN PERSON' 27 | ) 28 | or 29 | ( 30 | p_partkey = l_partkey 31 | and p_brand = 'Brand#52' 32 | and p_container in ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG') 33 | and l_quantity >= 30 and l_quantity <= 30 + 10 34 | and p_size between 1 and 15 35 | and l_shipmode in ('AIR', 'AIR REG') 36 | and l_shipinstruct = 'DELIVER IN PERSON' 37 | ); 38 | -------------------------------------------------------------------------------- /testdata/queries/q2.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 2 derived from TPC-H query 2 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | s_acctbal, 5 | s_name, 6 | n_name, 7 | p_partkey, 8 | p_mfgr, 9 | s_address, 10 | s_phone, 11 | s_comment 12 | from 13 | part, 14 | supplier, 15 | partsupp, 16 | nation, 17 | region 18 | where 19 | p_partkey = ps_partkey 20 | and s_suppkey = ps_suppkey 21 | and p_size = 48 22 | and p_type like '%TIN' 23 | and s_nationkey = n_nationkey 24 | and n_regionkey = r_regionkey 25 | and r_name = 'ASIA' 26 | and ps_supplycost = ( 27 | select 28 | min(ps_supplycost) 29 | from 30 | partsupp, 31 | supplier, 32 | nation, 33 | region 34 | where 35 | p_partkey = ps_partkey 36 | and s_suppkey = ps_suppkey 37 | and s_nationkey = n_nationkey 38 | and n_regionkey = r_regionkey 39 | and r_name = 'ASIA' 40 | ) 41 | order by 42 | s_acctbal desc, 43 | n_name, 44 | s_name, 45 | p_partkey limit 100; 46 | -------------------------------------------------------------------------------- /testdata/queries/q20.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 20 derived from TPC-H query 20 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | s_name, 5 | s_address 6 | from 7 | supplier, 8 | nation 9 | where 10 | s_suppkey in ( 11 | select 12 | ps_suppkey 13 | from 14 | partsupp 15 | where 16 | ps_partkey in ( 17 | select 18 | p_partkey 19 | from 20 | part 21 | where 22 | p_name like 'blanched%' 23 | ) 24 | and ps_availqty > ( 25 | select 26 | 0.5 * sum(l_quantity) 27 | from 28 | lineitem 29 | where 30 | l_partkey = ps_partkey 31 | and l_suppkey = ps_suppkey 32 | and l_shipdate >= date '1993-01-01' 33 | and l_shipdate < date '1993-01-01' + interval '1' year 34 | ) 35 | ) 36 | and s_nationkey = n_nationkey 37 | and n_name = 'KENYA' 38 | order by 39 | s_name; 40 | -------------------------------------------------------------------------------- /testdata/queries/q21.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 21 derived from TPC-H query 21 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | s_name, 5 | count(*) as numwait 6 | from 7 | supplier, 8 | lineitem l1, 9 | orders, 10 | nation 11 | where 12 | s_suppkey = l1.l_suppkey 13 | and o_orderkey = l1.l_orderkey 14 | and o_orderstatus = 'F' 15 | and l1.l_receiptdate > l1.l_commitdate 16 | and exists ( 17 | select 18 | * 19 | from 20 | lineitem l2 21 | where 22 | l2.l_orderkey = l1.l_orderkey 23 | and l2.l_suppkey <> l1.l_suppkey 24 | ) 25 | and not exists ( 26 | select 27 | * 28 | from 29 | lineitem l3 30 | where 31 | l3.l_orderkey = l1.l_orderkey 32 | and l3.l_suppkey <> l1.l_suppkey 33 | and l3.l_receiptdate > l3.l_commitdate 34 | ) 35 | and s_nationkey = n_nationkey 36 | and n_name = 'ARGENTINA' 37 | group by 38 | s_name 39 | order by 40 | numwait desc, 41 | s_name limit 100; 42 | -------------------------------------------------------------------------------- /testdata/queries/q22.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 22 derived from TPC-H query 22 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | cntrycode, 5 | count(*) as numcust, 6 | sum(c_acctbal) as totacctbal 7 | from 8 | ( 9 | select 10 | substring(c_phone from 1 for 2) as cntrycode, 11 | c_acctbal 12 | from 13 | customer 14 | where 15 | substring(c_phone from 1 for 2) in 16 | ('24', '34', '16', '30', '33', '14', '13') 17 | and c_acctbal > ( 18 | select 19 | avg(c_acctbal) 20 | from 21 | customer 22 | where 23 | c_acctbal > 0.00 24 | and substring(c_phone from 1 for 2) in 25 | ('24', '34', '16', '30', '33', '14', '13') 26 | ) 27 | and not exists ( 28 | select 29 | * 30 | from 31 | orders 32 | where 33 | o_custkey = c_custkey 34 | ) 35 | ) as custsale 36 | group by 37 | cntrycode 38 | order by 39 | cntrycode; 40 | -------------------------------------------------------------------------------- /testdata/queries/q3.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 3 derived from TPC-H query 3 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | l_orderkey, 5 | sum(l_extendedprice * (1 - l_discount)) as revenue, 6 | o_orderdate, 7 | o_shippriority 8 | from 9 | customer, 10 | orders, 11 | lineitem 12 | where 13 | c_mktsegment = 'BUILDING' 14 | and c_custkey = o_custkey 15 | and l_orderkey = o_orderkey 16 | and o_orderdate < date '1995-03-15' 17 | and l_shipdate > date '1995-03-15' 18 | group by 19 | l_orderkey, 20 | o_orderdate, 21 | o_shippriority 22 | order by 23 | revenue desc, 24 | o_orderdate limit 10; 25 | -------------------------------------------------------------------------------- /testdata/queries/q4.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 4 derived from TPC-H query 4 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | o_orderpriority, 5 | count(*) as order_count 6 | from 7 | orders 8 | where 9 | o_orderdate >= date '1995-04-01' 10 | and o_orderdate < date '1995-04-01' + interval '3' month 11 | and exists ( 12 | select 13 | * 14 | from 15 | lineitem 16 | where 17 | l_orderkey = o_orderkey 18 | and l_commitdate < l_receiptdate 19 | ) 20 | group by 21 | o_orderpriority 22 | order by 23 | o_orderpriority; 24 | -------------------------------------------------------------------------------- /testdata/queries/q5.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 5 derived from TPC-H query 5 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | n_name, 5 | sum(l_extendedprice * (1 - l_discount)) as revenue 6 | from 7 | customer, 8 | orders, 9 | lineitem, 10 | supplier, 11 | nation, 12 | region 13 | where 14 | c_custkey = o_custkey 15 | and l_orderkey = o_orderkey 16 | and l_suppkey = s_suppkey 17 | and c_nationkey = s_nationkey 18 | and s_nationkey = n_nationkey 19 | and n_regionkey = r_regionkey 20 | and r_name = 'AFRICA' 21 | and o_orderdate >= date '1994-01-01' 22 | and o_orderdate < date '1994-01-01' + interval '1' year 23 | group by 24 | n_name 25 | order by 26 | revenue desc; 27 | -------------------------------------------------------------------------------- /testdata/queries/q6.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 6 derived from TPC-H query 6 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | sum(l_extendedprice * l_discount) as revenue 5 | from 6 | lineitem 7 | where 8 | l_shipdate >= date '1994-01-01' 9 | and l_shipdate < date '1994-01-01' + interval '1' year 10 | and l_discount between 0.04 - 0.01 and 0.04 + 0.01 11 | and l_quantity < 24; 12 | -------------------------------------------------------------------------------- /testdata/queries/q7.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 7 derived from TPC-H query 7 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | supp_nation, 5 | cust_nation, 6 | l_year, 7 | sum(volume) as revenue 8 | from 9 | ( 10 | select 11 | n1.n_name as supp_nation, 12 | n2.n_name as cust_nation, 13 | extract(year from l_shipdate) as l_year, 14 | l_extendedprice * (1 - l_discount) as volume 15 | from 16 | supplier, 17 | lineitem, 18 | orders, 19 | customer, 20 | nation n1, 21 | nation n2 22 | where 23 | s_suppkey = l_suppkey 24 | and o_orderkey = l_orderkey 25 | and c_custkey = o_custkey 26 | and s_nationkey = n1.n_nationkey 27 | and c_nationkey = n2.n_nationkey 28 | and ( 29 | (n1.n_name = 'GERMANY' and n2.n_name = 'IRAQ') 30 | or (n1.n_name = 'IRAQ' and n2.n_name = 'GERMANY') 31 | ) 32 | and l_shipdate between date '1995-01-01' and date '1996-12-31' 33 | ) as shipping 34 | group by 35 | supp_nation, 36 | cust_nation, 37 | l_year 38 | order by 39 | supp_nation, 40 | cust_nation, 41 | l_year; 42 | -------------------------------------------------------------------------------- /testdata/queries/q8.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 8 derived from TPC-H query 8 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | o_year, 5 | sum(case 6 | when nation = 'IRAQ' then volume 7 | else 0 8 | end) / sum(volume) as mkt_share 9 | from 10 | ( 11 | select 12 | extract(year from o_orderdate) as o_year, 13 | l_extendedprice * (1 - l_discount) as volume, 14 | n2.n_name as nation 15 | from 16 | part, 17 | supplier, 18 | lineitem, 19 | orders, 20 | customer, 21 | nation n1, 22 | nation n2, 23 | region 24 | where 25 | p_partkey = l_partkey 26 | and s_suppkey = l_suppkey 27 | and l_orderkey = o_orderkey 28 | and o_custkey = c_custkey 29 | and c_nationkey = n1.n_nationkey 30 | and n1.n_regionkey = r_regionkey 31 | and r_name = 'MIDDLE EAST' 32 | and s_nationkey = n2.n_nationkey 33 | and o_orderdate between date '1995-01-01' and date '1996-12-31' 34 | and p_type = 'LARGE PLATED STEEL' 35 | ) as all_nations 36 | group by 37 | o_year 38 | order by 39 | o_year; 40 | -------------------------------------------------------------------------------- /testdata/queries/q9.sql: -------------------------------------------------------------------------------- 1 | -- SQLBench-H query 9 derived from TPC-H query 9 under the terms of the TPC Fair Use Policy. 2 | -- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. 3 | select 4 | nation, 5 | o_year, 6 | sum(amount) as sum_profit 7 | from 8 | ( 9 | select 10 | n_name as nation, 11 | extract(year from o_orderdate) as o_year, 12 | l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount 13 | from 14 | part, 15 | supplier, 16 | lineitem, 17 | partsupp, 18 | orders, 19 | nation 20 | where 21 | s_suppkey = l_suppkey 22 | and ps_suppkey = l_suppkey 23 | and ps_partkey = l_partkey 24 | and p_partkey = l_partkey 25 | and o_orderkey = l_orderkey 26 | and s_nationkey = n_nationkey 27 | and p_name like '%moccasin%' 28 | ) as profit 29 | group by 30 | nation, 31 | o_year 32 | order by 33 | nation, 34 | o_year desc; 35 | --------------------------------------------------------------------------------