├── .asf.yaml ├── .cargo └── config.toml ├── .dockerignore ├── .github └── workflows │ ├── build.yml │ ├── k8s.yml │ └── main.yml ├── .gitignore ├── .pre-commit-config.yaml ├── Cargo.lock ├── Cargo.toml ├── LICENSE.txt ├── NOTICE ├── README.md ├── build.rs ├── datafusion_ray ├── __init__.py ├── core.py ├── friendly.py └── util.py ├── dev ├── create_license.py └── release │ ├── README.md │ ├── check-rat-report.py │ ├── create-tarball.sh │ ├── generate-changelog.py │ ├── rat_exclude_files.txt │ ├── release-tarball.sh │ ├── run-rat.sh │ └── verify-release-candidate.sh ├── docs ├── README.md ├── benchmarks │ └── df-ray-0.1.0rc1 │ │ ├── i4g.4xlarge │ │ ├── i4g.4xlarge_s3 │ │ ├── m7g.12x │ │ ├── m7g.12xlarge │ │ ├── m7g.12xlarge_s3 │ │ ├── m7gd.12x │ │ ├── m7gd.12xlarge │ │ └── r8g.8xlarge └── contributing.md ├── examples ├── http_csv.py └── tips.py ├── k8s ├── README.md ├── bench_toolbox.py ├── cmds.py ├── machine_prep.sh ├── pricing.py ├── pvcs.yaml.template ├── ray_cluster.yaml.template ├── ray_job.sh.template ├── requirements.txt.template ├── spark_job.yaml.template └── spark_tpcbench.py ├── pyproject.toml ├── src ├── codec.rs ├── context.rs ├── dataframe.rs ├── flight.rs ├── isolator.rs ├── lib.rs ├── max_rows.rs ├── physical.rs ├── pre_fetch.rs ├── processor_service.rs ├── proto │ ├── datafusion.proto │ ├── datafusion_common.proto │ ├── datafusion_ray.proto │ ├── generated │ │ ├── mod.rs │ │ └── protobuf.rs │ └── mod.rs ├── stage.rs ├── stage_reader.rs └── util.rs ├── testdata ├── tips │ └── tips.parquet └── tpch │ └── .gitignore └── tpch ├── make_data.py ├── 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 ├── requirements.txt └── tpcbench.py /.asf.yaml: -------------------------------------------------------------------------------- 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 | notifications: 19 | commits: commits@datafusion.apache.org 20 | issues: github@datafusion.apache.org 21 | pullrequests: github@datafusion.apache.org 22 | jira_options: link label worklog 23 | github: 24 | description: "Apache DataFusion Ray" 25 | homepage: https://datafusion.apache.org/ray 26 | enabled_merge_buttons: 27 | squash: true 28 | merge: false 29 | rebase: false 30 | features: 31 | issues: true 32 | 33 | staging: 34 | whoami: asf-staging 35 | subdir: ray 36 | 37 | publish: 38 | whoami: asf-site 39 | subdir: ray 40 | -------------------------------------------------------------------------------- /.cargo/config.toml: -------------------------------------------------------------------------------- 1 | [target.x86_64-apple-darwin] 2 | rustflags = ["-C", "link-arg=-undefined", "-C", "link-arg=dynamic_lookup"] 3 | 4 | [target.aarch64-apple-darwin] 5 | rustflags = ["-C", "link-arg=-undefined", "-C", "link-arg=dynamic_lookup"] 6 | 7 | [build] 8 | rustflags = ["-C", "target-cpu=native"] 9 | -------------------------------------------------------------------------------- /.dockerignore: -------------------------------------------------------------------------------- 1 | # Ignore the target directory where Rust build artifacts are placed 2 | /target 3 | 4 | # Ignore the local Cargo registry and git db (cached crates) 5 | /cargo/.git 6 | /cargo/registry 7 | 8 | # Ignore build scripts or binaries 9 | *.exe 10 | *.dll 11 | *.so 12 | *.dylib 13 | 14 | # Ignore backup and swap files 15 | *.swp 16 | *.swo 17 | *.bak 18 | 19 | # Ignore any IDE-specific or editor-specific files (e.g., VSCode, IntelliJ, etc.) 20 | **/.vscode 21 | **/.idea 22 | **/.gdb_history 23 | **/.git 24 | **/.DS_Store 25 | 26 | # Ignore test result files 27 | /tests/results/ 28 | 29 | # Ignore unnecessary temporary or system files 30 | **/node_modules 31 | *.log 32 | *.tmp 33 | *.lock 34 | !Cargo.lock 35 | -------------------------------------------------------------------------------- /.github/workflows/k8s.yml: -------------------------------------------------------------------------------- 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 | name: Kubernetes 18 | 19 | on: 20 | # push: 21 | # pull_request: 22 | workflow_dispatch: 23 | 24 | env: 25 | RAY_HELM_VERSION: 1.1.0 26 | PYTHON_VERSION: 3.9 27 | IMAGE_REPOSITORY: ghcr.io/apache/datafusion-ray 28 | 29 | jobs: 30 | build: 31 | runs-on: ubuntu-latest 32 | 33 | steps: 34 | - uses: actions/checkout@v3 35 | 36 | - name: Docker meta 37 | id: meta 38 | uses: docker/metadata-action@v5 39 | with: 40 | images: | 41 | ${{ env.IMAGE_REPOSITORY }} 42 | tags: | 43 | type=ref,event=branch 44 | type=ref,event=pr 45 | type=semver,pattern={{version}} 46 | type=semver,pattern={{major}}.{{minor}} 47 | type=semver,pattern={{major}} 48 | type=sha 49 | 50 | - name: Set up Docker Buildx 51 | uses: docker/setup-buildx-action@v3 52 | 53 | - name: Log in to GitHub Container Registry 54 | if: startsWith(github.ref, 'refs/tags/') 55 | uses: docker/login-action@v3 56 | with: 57 | registry: ghcr.io 58 | username: ${{ github.actor }} 59 | password: ${{ secrets.GITHUB_TOKEN }} 60 | 61 | - name: Build and push Docker image 62 | uses: docker/build-push-action@v6 63 | with: 64 | context: . 65 | file: ./k8s/Dockerfile 66 | tags: ${{ steps.meta.outputs.tags }} 67 | labels: ${{ steps.meta.outputs.labels }} 68 | push: ${{ startsWith(github.ref, 'refs/tags/') }} 69 | load: true 70 | 71 | - name: Kind Cluster 72 | uses: helm/kind-action@v1.10.0 73 | with: 74 | config: ./k8s/kind-config.yaml 75 | 76 | - name: Free Disk Space (Ubuntu) 77 | uses: jlumbroso/free-disk-space@v1.3.1 78 | with: 79 | # this might remove tools that are actually needed, 80 | # when set to "true" but frees about 6 GB 81 | tool-cache: true 82 | docker-images: false 83 | 84 | - name: Extract short SHA 85 | run: | 86 | SHORT_SHA=$(echo "${GITHUB_SHA}" | cut -c1-7) 87 | echo "IMAGE_TAG=sha-${SHORT_SHA}" >> "$GITHUB_ENV" 88 | echo "Extracted short SHA tag: sha-${SHORT_SHA}" 89 | 90 | - name: Load Docker image into Kind cluster 91 | run: | 92 | kind load docker-image ${{ env.IMAGE_REPOSITORY }}:${{ env.IMAGE_TAG }} -n chart-testing 93 | 94 | - name: Install Helm 95 | uses: azure/setup-helm@v4.2.0 96 | with: 97 | version: "3.16.0" 98 | 99 | - name: Deploy helm chart 100 | run: | 101 | helm repo add kuberay https://ray-project.github.io/kuberay-helm/ 102 | helm install kuberay-operator kuberay/kuberay-operator --version ${{ env.RAY_HELM_VERSION }} 103 | helm install raycluster kuberay/ray-cluster --version ${{ env.RAY_HELM_VERSION }} \ 104 | --set image.repository=${{ env.IMAGE_REPOSITORY }} \ 105 | --set image.tag=${{ env.IMAGE_TAG }} \ 106 | --set imagePullPolicy=Always 107 | echo "Deployed Ray cluster with image repository: ${{ env.IMAGE_REPOSITORY }} and tag: ${{ env.IMAGE_TAG }}" 108 | 109 | - name: Set up Python 110 | uses: actions/setup-python@v3 111 | with: 112 | python-version: ${{ env.PYTHON_VERSION }} 113 | 114 | - name: Install python dependencies 115 | run: | 116 | python -m pip install --upgrade pip 117 | pip install -r requirements-in.txt 118 | 119 | - name: Submit an example ray Job 120 | run: | 121 | kubectl port-forward service/raycluster-kuberay-head-svc 8265:8265 & 122 | export RAY_ADDRESS="http://127.0.0.1:8265" 123 | pip install "ray[default]" 124 | ray job submit --working-dir ./examples/ -- python3 tips.py 125 | -------------------------------------------------------------------------------- /.github/workflows/main.yml: -------------------------------------------------------------------------------- 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 | name: Python Tests 19 | on: 20 | push: 21 | branches: [main] 22 | pull_request: 23 | branches: [main] 24 | workflow_dispatch: 25 | 26 | concurrency: 27 | group: ${{ github.repository }}-${{ github.head_ref || github.sha }}-${{ github.workflow }} 28 | cancel-in-progress: true 29 | 30 | env: 31 | MATURIN_PEP517_ARGS: --profile=dev 32 | 33 | jobs: 34 | validate-tpch: 35 | runs-on: ${{ matrix.platform.image }} 36 | strategy: 37 | fail-fast: false 38 | matrix: 39 | platform: 40 | - image: ubuntu-latest 41 | target: x86_64-unknown-linux-musl 42 | python-version: 43 | - "3.10" 44 | - "3.11" 45 | - "3.12" 46 | ray-version: 47 | - "2.40" 48 | - "2.41" 49 | - "2.42.1" 50 | - "2.43" 51 | steps: 52 | - uses: actions/checkout@v4 53 | 54 | - name: Setup Rust Toolchain 55 | uses: dtolnay/rust-toolchain@stable 56 | id: rust-toolchain 57 | with: 58 | target: ${{ matrix.platform.target }} 59 | 60 | - name: Check Rust output 61 | id: rust-toolchain-out 62 | run: | 63 | rustc --version --verbose 64 | cargo --version 65 | 66 | - name: Install Protoc 67 | uses: arduino/setup-protoc@v3 68 | with: 69 | version: "27.4" 70 | repo-token: ${{ secrets.GITHUB_TOKEN }} 71 | 72 | - name: Cache Cargo 73 | uses: actions/cache@v4 74 | with: 75 | path: ~/.cargo 76 | key: cargo-cache-${{ steps.rust-toolchain.outputs.cachekey }}-${{ hashFiles('Cargo.lock') }} 77 | 78 | - name: install uv 79 | uses: astral-sh/setup-uv@v5 80 | with: 81 | enable-cache: true 82 | python-version: ${{ matrix.python-version }} 83 | 84 | - name: initial project sync 85 | run: | 86 | cargo --version 87 | uv sync --no-install-package datafusion-ray --no-install-package ray 88 | 89 | # reset the version of ray in pyproject.toml 90 | # to agree align with our matrix version 91 | - name: add ray 92 | run: | 93 | cargo --version 94 | uv add --no-sync 'ray[default]==${{ matrix.ray-version}}' 95 | 96 | # sync the environment, building everything other than datafusion-ray 97 | - name: install dependencies 98 | run: | 99 | uv sync --no-install-package datafusion-ray 100 | working-directory: ${{ github.workspace }} 101 | 102 | # build datafusion ray honoring the MATURIN_PEP517_ARGS env 103 | # var to ensure that we build this in dev mode so its faster 104 | - name: maturin develop 105 | run: uv run maturin develop --uv 106 | working-directory: ${{ github.workspace }} 107 | 108 | - name: Cache the generated dataset 109 | id: cache-tpch-dataset 110 | uses: actions/cache@v4 111 | with: 112 | path: ./testdata/tpch 113 | key: tpch-data 114 | 115 | - name: create the dataset 116 | if: ${{ steps.cache-tpch-dataset.outputs.cache-hit != 'true' }} 117 | run: | 118 | uv run tpch/make_data.py 1 testdata/tpch/ 119 | 120 | # run the tpcbench.py file with --validate which will cause 121 | # it to exit with an error code if any of the queries do not validate 122 | - name: validate tpch 123 | env: 124 | DATAFUSION_RAY_LOG_LEVEL: debug 125 | RAY_COLOR_PREFIX: 1 126 | RAY_DEDUP_LOGS: 0 127 | RUST_BACKTRACE: 1 128 | run: | 129 | uv run python tpch/tpcbench.py \ 130 | --data='file:///${{ github.workspace }}/testdata/tpch/' \ 131 | --concurrency 3 \ 132 | --partitions-per-processor 2 \ 133 | --batch-size=8192 \ 134 | --processor-pool-min=20 \ 135 | --validate 136 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | .idea 2 | target 3 | __pycache__ 4 | venv 5 | *.so 6 | *.log 7 | results-sf* 8 | data 9 | tpch/tpch-dbgen 10 | -------------------------------------------------------------------------------- /.pre-commit-config.yaml: -------------------------------------------------------------------------------- 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 | repos: 19 | - repo: https://github.com/rhysd/actionlint 20 | rev: v1.6.23 21 | hooks: 22 | - id: actionlint-docker 23 | - repo: https://github.com/astral-sh/ruff-pre-commit 24 | # Ruff version. 25 | rev: v0.3.0 26 | hooks: 27 | # Run the linter. 28 | - id: ruff 29 | # Run the formatter. 30 | - id: ruff-format 31 | - repo: local 32 | hooks: 33 | - id: rust-fmt 34 | name: Rust fmt 35 | description: Run cargo fmt on files included in the commit. rustfmt should be installed before-hand. 36 | entry: cargo fmt --all -- 37 | pass_filenames: true 38 | types: [file, rust] 39 | language: system 40 | - id: rust-clippy 41 | name: Rust clippy 42 | description: Run cargo clippy on files included in the commit. clippy should be installed before-hand. 43 | entry: cargo clippy --all-targets --all-features -- -Dclippy::all -Aclippy::redundant_closure 44 | pass_filenames: false 45 | types: [file, rust] 46 | language: system 47 | 48 | default_language_version: 49 | python: python3 50 | -------------------------------------------------------------------------------- /Cargo.toml: -------------------------------------------------------------------------------- 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 | [package] 19 | name = "datafusion_ray" 20 | description = "DataFusion on Ray" 21 | homepage = "https://github.com/apache/datafusion-ray" 22 | repository = "https://github.com/apache/datafusion-ray" 23 | authors = ["Apache DataFusion "] 24 | version = "0.1.0-rc2" 25 | edition = "2024" 26 | readme = "README.md" 27 | license = "Apache-2.0" 28 | rust-version = "1.85" 29 | build = "build.rs" 30 | 31 | [dependencies] 32 | anyhow = "1" 33 | arrow = { version = "54", features = ["pyarrow", "ipc"] } 34 | arrow-flight = "54" 35 | async-stream = "0.3" 36 | async-channel = "2.3" 37 | bytesize = "1.3" 38 | datafusion = { version = "45", features = ["pyarrow", "avro"] } 39 | datafusion-python = { version = "45" } 40 | datafusion-proto = "45" 41 | env_logger = "0.11" 42 | futures = "0.3" 43 | glob = "0.3.1" 44 | itertools = "0.14" 45 | local-ip-address = "0.6" 46 | log = "0.4" 47 | object_store = { version = "0.11.0", features = [ 48 | "aws", 49 | "gcp", 50 | "azure", 51 | "http", 52 | ] } 53 | parking_lot = { version = "0.12", features = ["deadlock_detection"] } 54 | prost = "0.13" 55 | protobuf-src = "2.1" 56 | pyo3 = { version = "0.23", features = [ 57 | "extension-module", 58 | "abi3", 59 | "abi3-py38", 60 | ] } 61 | pyo3-async-runtimes = { version = "0.23", features = ["tokio-runtime"] } 62 | pyo3-pylogger = "0.3.0" 63 | rust_decimal = "1.36" 64 | tokio = { version = "1.43", features = [ 65 | "macros", 66 | "rt", 67 | "rt-multi-thread", 68 | "sync", 69 | "time", 70 | ] } 71 | tokio-stream = "0.1" 72 | 73 | tonic = { version = "0.12.3", default-features = false, features = [ 74 | "transport", 75 | "codegen", 76 | "prost", 77 | ] } 78 | uuid = "1.11.0" 79 | url = "2" 80 | 81 | [build-dependencies] 82 | prost-types = "0.13" 83 | rustc_version = "0.4.0" 84 | tonic-build = { version = "0.8", default-features = false, features = [ 85 | "transport", 86 | "prost", 87 | ] } 88 | url = "2" 89 | 90 | [dev-dependencies] 91 | tempfile = "3.17" 92 | 93 | [lib] 94 | name = "datafusion_ray" 95 | crate-type = ["cdylib", "rlib"] 96 | 97 | [package.metadata.maturin] 98 | name = "datafusion_ray._datafusion_ray_internal" 99 | 100 | [profile.release] 101 | lto = "thin" 102 | codegen-units = 1 103 | opt-level = 3 104 | debug = 0 105 | 106 | [profile.dev] 107 | opt-level = 1 108 | 109 | [profile.dev.package."*"] 110 | opt-level = 1 111 | -------------------------------------------------------------------------------- /NOTICE: -------------------------------------------------------------------------------- 1 | This product includes software developed at 2 | The Apache Software Foundation (http://www.apache.org/). -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | 19 | 20 | # DataFusion for Ray 21 | 22 | [![Apache licensed][license-badge]][license-url] 23 | [![Python Tests][actions-badge]][actions-url] 24 | [![Discord chat][discord-badge]][discord-url] 25 | 26 | [license-badge]: https://img.shields.io/badge/license-Apache%20v2-blue.svg 27 | [license-url]: https://github.com/apache/datafusion-ray/blob/main/LICENSE.txt 28 | [actions-badge]: https://github.com/apache/datafusion-ray/actions/workflows/main.yml/badge.svg 29 | [actions-url]: https://github.com/apache/datafusion-ray/actions?query=branch%3Amain 30 | [discord-badge]: https://img.shields.io/badge/Chat-Discord-purple 31 | [discord-url]: https://discord.com/invite/Qw5gKqHxUM 32 | 33 | ## Overview 34 | 35 | DataFusion for Ray is a distributed execution framework that enables DataFusion DataFrame and SQL queries to run on a 36 | Ray cluster. This integration allows users to leverage Ray's dynamic scheduling capabilities while executing 37 | queries in a distributed fashion. 38 | 39 | ## Execution Modes 40 | 41 | DataFusion for Ray supports two execution modes: 42 | 43 | ### Streaming Execution 44 | 45 | This mode mimics the default execution strategy of DataFusion. Each operator in the query plan starts executing 46 | as soon as its inputs are available, leading to a more pipelined execution model. 47 | 48 | ### Batch Execution 49 | 50 | _Note: Batch Execution is not implemented yet. Tracking issue: _ 51 | 52 | In this mode, execution follows a staged model similar to Apache Spark. Each query stage runs to completion, producing 53 | intermediate shuffle files that are persisted and used as input for the next stage. 54 | 55 | ## Getting Started 56 | 57 | See the [contributor guide] for instructions on building DataFusion for Ray. 58 | 59 | Once installed, you can run queries using DataFusion's familiar API while leveraging the distributed execution 60 | capabilities of Ray. 61 | 62 | ```python 63 | # from example in ./examples/http_csv.py 64 | import ray 65 | from datafusion_ray import DFRayContext, df_ray_runtime_env 66 | 67 | ray.init(runtime_env=df_ray_runtime_env) 68 | 69 | ctx = DFRayContext() 70 | ctx.register_csv( 71 | "aggregate_test_100", 72 | "https://github.com/apache/arrow-testing/raw/master/data/csv/aggregate_test_100.csv", 73 | ) 74 | 75 | df = ctx.sql("SELECT c1,c2,c3 FROM aggregate_test_100 LIMIT 5") 76 | 77 | df.show() 78 | ``` 79 | 80 | ## Contributing 81 | 82 | Contributions are welcome! Please open an issue or submit a pull request if you would like to contribute. See the 83 | [contributor guide] for more information. 84 | 85 | ## License 86 | 87 | DataFusion for Ray is licensed under Apache 2.0. 88 | 89 | [contributor guide]: docs/contributing.md 90 | -------------------------------------------------------------------------------- /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/datafusion_ray.proto").exists() { 36 | println!("cargo:rerun-if-changed=src/proto/datafusion_common.proto"); 37 | println!("cargo:rerun-if-changed=src/proto/datafusion.proto"); 38 | println!("cargo:rerun-if-changed=src/proto/datafusion_ray.proto"); 39 | tonic_build::configure() 40 | .extern_path(".datafusion", "::datafusion_proto::protobuf") 41 | .extern_path(".datafusion_common", "::datafusion_proto::protobuf") 42 | .compile(&["src/proto/datafusion_ray.proto"], &["src/proto"]) 43 | .map_err(|e| format!("protobuf compilation failed: {e}"))?; 44 | let generated_source_path = out.join("datafusion_ray.protobuf.rs"); 45 | let code = std::fs::read_to_string(generated_source_path).unwrap(); 46 | let mut file = std::fs::OpenOptions::new() 47 | .write(true) 48 | .truncate(true) 49 | .create(true) 50 | .open(path) 51 | .unwrap(); 52 | file.write_all(code.as_str().as_ref()).unwrap(); 53 | } 54 | 55 | Ok(()) 56 | } 57 | -------------------------------------------------------------------------------- /datafusion_ray/__init__.py: -------------------------------------------------------------------------------- 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 | try: 19 | import importlib.metadata as importlib_metadata 20 | except ImportError: 21 | import importlib_metadata 22 | 23 | from .core import DFRayContext, df_ray_runtime_env 24 | 25 | from . import util 26 | 27 | __all__ = ["DFRayContext", "df_ray_runtime_env", "util"] 28 | 29 | __version__ = importlib_metadata.version(__name__) 30 | -------------------------------------------------------------------------------- /datafusion_ray/util.py: -------------------------------------------------------------------------------- 1 | from datafusion_ray._datafusion_ray_internal import ( 2 | LocalValidator, 3 | prettify, 4 | ) 5 | -------------------------------------------------------------------------------- /dev/release/README.md: -------------------------------------------------------------------------------- 1 | 19 | 20 | # DataFusion for Ray Release Process 21 | 22 | Development happens on the `main` branch, and most of the time, we depend on DataFusion using GitHub dependencies 23 | rather than using an official release from crates.io. This allows us to pick up new features and bug fixes frequently 24 | by creating PRs to move to a later revision of the code. It also means we can incrementally make updates that are 25 | required due to changes in DataFusion rather than having a large amount of work to do when the next official release 26 | is available. 27 | 28 | When there is a new official release of DataFusion, we update the `main` branch to point to that, update the version 29 | number, and create a new release branch, such as `branch-0.2`. Once this branch is created, we switch the `main` branch 30 | back to using GitHub dependencies. The release activity (such as generating the changelog) can then happen on the 31 | release branch without blocking ongoing development in the `main` branch. 32 | 33 | We can cherry-pick commits from the `main` branch into `branch-0.2` as needed and then create new patch releases 34 | from that branch. 35 | 36 | ## Detailed Guide 37 | 38 | ### Pre-requisites 39 | 40 | Releases can currently only be created by PMC members due to the permissions needed. 41 | 42 | You will need a GitHub Personal Access Token. Follow 43 | [these instructions](https://docs.github.com/en/authentication/keeping-your-account-and-data-secure/creating-a-personal-access-token) 44 | to generate one if you do not already have one. 45 | 46 | You will need a PyPI API token. Create one at , setting the “Scope” to 47 | “Entire account”. 48 | 49 | You will also need access to the [datafusion-ray](https://test.pypi.org/project/datafusion-ray/) project on testpypi. 50 | 51 | ### Preparing the `main` Branch 52 | 53 | Before creating a new release: 54 | 55 | - We need to ensure that the main branch does not have any GitHub dependencies 56 | - a PR should be created and merged to update the major version number of the project 57 | - A new release branch should be created, such as `branch-0.2` 58 | 59 | ### Change Log 60 | 61 | We maintain a `CHANGELOG.md` so our users know what has been changed between releases. 62 | 63 | The changelog is generated using a Python script: 64 | 65 | ```bash 66 | GITHUB_TOKEN= ./dev/release/generate-changelog.py 0.1.0 HEAD 0.2.0 > dev/changelog/0.2.0.md 67 | ``` 68 | 69 | This script creates a changelog from GitHub PRs based on the labels associated with them as well as looking for 70 | titles starting with `feat:`, `fix:`, or `docs:` . The script will produce output similar to: 71 | 72 | ``` 73 | Fetching list of commits between 24.0.0 and HEAD 74 | Fetching pull requests 75 | Categorizing pull requests 76 | Generating changelog content 77 | ``` 78 | 79 | ### Preparing a Release Candidate 80 | 81 | ### Tag the Repository 82 | 83 | ```bash 84 | git tag 0.2.0-rc1 85 | git push apache 0.2.0-rc1 86 | ``` 87 | 88 | ### Create a source release 89 | 90 | ```bash 91 | ./dev/release/create-tarball.sh 0.2.0 1 92 | ``` 93 | 94 | This will also create the email template to send to the mailing list. 95 | 96 | Create a draft email using this content, but do not send until after completing the next step. 97 | 98 | ### Publish Python Artifacts to testpypi 99 | 100 | This section assumes some familiarity with publishing Python packages to PyPi. For more information, refer to \ 101 | [this tutorial](https://packaging.python.org/en/latest/tutorials/packaging-projects/#uploading-the-distribution-archives). 102 | 103 | #### Publish Python Wheels to testpypi 104 | 105 | Pushing an `rc` tag to the release branch will cause a GitHub Workflow to run that will build the Python wheels. 106 | 107 | Go to and look for an action named "Python Release Build" 108 | that has run against the pushed tag. 109 | 110 | Click on the action and scroll down to the bottom of the page titled "Artifacts". Download `dist.zip`. It should 111 | contain files such as: 112 | 113 | ```text 114 | datafusion-ray-0.2.0-cp37-abi3-macosx_10_7_x86_64.whl 115 | datafusion-ray-0.2.0-cp37-abi3-macosx_11_0_arm64.whl 116 | datafusion-ray-0.2.0-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl 117 | datafusion-ray-0.2.0-cp37-abi3-win_amd64.whl 118 | ``` 119 | 120 | Upload the wheels to testpypi. 121 | 122 | ```bash 123 | unzip dist.zip 124 | python3 -m pip install --upgrade setuptools twine build 125 | python3 -m twine upload --repository testpypi datafusion-ray-0.2.0-cp37-abi3-*.whl 126 | ``` 127 | 128 | When prompted for username, enter `__token__`. When prompted for a password, enter a valid GitHub Personal Access Token 129 | 130 | #### Publish Python Source Distribution to testpypi 131 | 132 | Download the source tarball created in the previous step, untar it, and run: 133 | 134 | ```bash 135 | maturin sdist 136 | ``` 137 | 138 | This will create a file named `dist/datafusion-ray-0.2.0.tar.gz`. Upload this to testpypi: 139 | 140 | ```bash 141 | python3 -m twine upload --repository testpypi dist/datafusion-ray-0.2.0.tar.gz 142 | ``` 143 | 144 | ### Send the Email 145 | 146 | Send the email to start the vote. 147 | 148 | ## Verifying a Release 149 | 150 | Running the unit tests against a testpypi release candidate: 151 | 152 | ```bash 153 | # clone a fresh repo 154 | git clone https://github.com/apache/datafusion-ray.git 155 | cd datafusion-ray 156 | 157 | # checkout the release commit 158 | git fetch --tags 159 | git checkout 0.2.0-rc1 160 | 161 | # create the env 162 | python3 -m venv venv 163 | source venv/bin/activate 164 | 165 | # install release candidate 166 | pip install --extra-index-url https://test.pypi.org/simple/ datafusion-ray==0.2.0 167 | 168 | # only dep needed to run tests is pytest 169 | pip install pytest 170 | 171 | # run the tests 172 | pytest --import-mode=importlib python/tests 173 | ``` 174 | 175 | Try running one of the examples from the top-level README, or write some custom Python code to query some available 176 | data files. 177 | 178 | ## Publishing a Release 179 | 180 | ### Publishing Apache Source Release 181 | 182 | Once the vote passes, we can publish the release. 183 | 184 | Create the source release tarball: 185 | 186 | ```bash 187 | ./dev/release/release-tarball.sh 0.2.0 1 188 | ``` 189 | 190 | ### Publishing Python Artifacts to PyPi 191 | 192 | Go to the Test PyPI page of Datafusion, and download 193 | [all published artifacts](https://test.pypi.org/project/datafusion-ray/#files) under `dist-release/` directory. Then proceed 194 | uploading them using `twine`: 195 | 196 | ```bash 197 | twine upload --repository pypi dist-release/* 198 | ``` 199 | 200 | ### Push the Release Tag 201 | 202 | ```bash 203 | git checkout 0.2.0-rc1 204 | git tag 0.2.0 205 | git push apache 0.2.0 206 | ``` 207 | 208 | ### Add the release to Apache Reporter 209 | 210 | Add the release to with a version name prefixed with `DATAFUSION-RAY`, 211 | for example `DATAFUSION-RAY-0.2.0`. 212 | 213 | The release information is used to generate a template for a board report (see example from Apache Arrow 214 | [here](https://github.com/apache/arrow/pull/14357)). 215 | 216 | ### Delete old RCs and Releases 217 | 218 | See the ASF documentation on [when to archive](https://www.apache.org/legal/release-policy.html#when-to-archive) 219 | for more information. 220 | 221 | #### Deleting old release candidates from `dev` svn 222 | 223 | Release candidates should be deleted once the release is published. 224 | 225 | Get a list of DataFusion for Ray release candidates: 226 | 227 | ```bash 228 | svn ls https://dist.apache.org/repos/dist/dev/datafusion | grep datafusion-ray 229 | ``` 230 | 231 | Delete a release candidate: 232 | 233 | ```bash 234 | svn delete -m "delete old DataFusion RC" https://dist.apache.org/repos/dist/dev/datafusion/apache-datafusion-ray-0.1.0-rc1/ 235 | ``` 236 | 237 | #### Deleting old releases from `release` svn 238 | 239 | Only the latest release should be available. Delete old releases after publishing the new release. 240 | 241 | Get a list of DataFusion releases: 242 | 243 | ```bash 244 | svn ls https://dist.apache.org/repos/dist/release/datafusion | grep datafusion-ray 245 | ``` 246 | 247 | Delete a release: 248 | 249 | ```bash 250 | svn delete -m "delete old DataFusion release" https://dist.apache.org/repos/dist/release/datafusion/datafusion-ray-0.1.0 251 | ``` 252 | -------------------------------------------------------------------------------- /dev/release/check-rat-report.py: -------------------------------------------------------------------------------- 1 | #!/usr/bin/python 2 | ############################################################################## 3 | # Licensed to the Apache Software Foundation (ASF) under one 4 | # or more contributor license agreements. See the NOTICE file 5 | # distributed with this work for additional information 6 | # regarding copyright ownership. The ASF licenses this file 7 | # to you under the Apache License, Version 2.0 (the 8 | # "License"); you may not use this file except in compliance 9 | # with the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, 14 | # software distributed under the License is distributed on an 15 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | # KIND, either express or implied. See the License for the 17 | # specific language governing permissions and limitations 18 | # under the License. 19 | ############################################################################## 20 | import fnmatch 21 | import re 22 | import sys 23 | import xml.etree.ElementTree as ET 24 | 25 | if len(sys.argv) != 3: 26 | sys.stderr.write( 27 | "Usage: %s exclude_globs.lst rat_report.xml\n" % sys.argv[0] 28 | ) 29 | sys.exit(1) 30 | 31 | exclude_globs_filename = sys.argv[1] 32 | xml_filename = sys.argv[2] 33 | 34 | globs = [line.strip() for line in open(exclude_globs_filename, "r")] 35 | 36 | tree = ET.parse(xml_filename) 37 | root = tree.getroot() 38 | resources = root.findall("resource") 39 | 40 | all_ok = True 41 | for r in resources: 42 | approvals = r.findall("license-approval") 43 | if not approvals or approvals[0].attrib["name"] == "true": 44 | continue 45 | clean_name = re.sub("^[^/]+/", "", r.attrib["name"]) 46 | excluded = False 47 | for g in globs: 48 | if fnmatch.fnmatch(clean_name, g): 49 | excluded = True 50 | break 51 | if not excluded: 52 | sys.stdout.write( 53 | "NOT APPROVED: %s (%s): %s\n" 54 | % (clean_name, r.attrib["name"], approvals[0].attrib["name"]) 55 | ) 56 | all_ok = False 57 | 58 | if not all_ok: 59 | sys.exit(1) 60 | 61 | print("OK") 62 | sys.exit(0) 63 | -------------------------------------------------------------------------------- /dev/release/create-tarball.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | # 3 | # Licensed to the Apache Software Foundation (ASF) under one 4 | # or more contributor license agreements. See the NOTICE file 5 | # distributed with this work for additional information 6 | # regarding copyright ownership. The ASF licenses this file 7 | # to you under the Apache License, Version 2.0 (the 8 | # "License"); you may not use this file except in compliance 9 | # with the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, 14 | # software distributed under the License is distributed on an 15 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | # KIND, either express or implied. See the License for the 17 | # specific language governing permissions and limitations 18 | # under the License. 19 | # 20 | 21 | # Adapted from https://github.com/apache/arrow-rs/tree/master/dev/release/create-tarball.sh 22 | 23 | # This script creates a signed tarball in 24 | # dev/dist/apache-datafusion-ray--.tar.gz and uploads it to 25 | # the "dev" area of the dist.apache.arrow repository and prepares an 26 | # email for sending to the dev@datafusion.apache.org list for a formal 27 | # vote. 28 | # 29 | # See release/README.md for full release instructions 30 | # 31 | # Requirements: 32 | # 33 | # 1. gpg setup for signing and have uploaded your public 34 | # signature to https://pgp.mit.edu/ 35 | # 36 | # 2. Logged into the apache svn server with the appropriate 37 | # credentials 38 | # 39 | # 3. Install the requests python package 40 | # 41 | # 42 | # Based in part on 02-source.sh from apache/arrow 43 | # 44 | 45 | set -e 46 | 47 | SOURCE_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)" 48 | SOURCE_TOP_DIR="$(cd "${SOURCE_DIR}/../../" && pwd)" 49 | 50 | if [ "$#" -ne 2 ]; then 51 | echo "Usage: $0 " 52 | echo "ex. $0 4.1.0 2" 53 | exit 54 | fi 55 | 56 | version=$1 57 | rc=$2 58 | tag="${version}-rc${rc}" 59 | 60 | echo "Attempting to create ${tarball} from tag ${tag}" 61 | release_hash=$(cd "${SOURCE_TOP_DIR}" && git rev-list --max-count=1 ${tag}) 62 | 63 | release=apache-datafusion-ray-${version} 64 | distdir=${SOURCE_TOP_DIR}/dev/dist/${release}-rc${rc} 65 | tarname=${release}.tar.gz 66 | tarball=${distdir}/${tarname} 67 | url="https://dist.apache.org/repos/dist/dev/datafusion/${release}-rc${rc}" 68 | 69 | if [ -z "$release_hash" ]; then 70 | echo "Cannot continue: unknown git tag: ${tag}" 71 | fi 72 | 73 | echo "Draft email for dev@datafusion.apache.org mailing list" 74 | echo "" 75 | echo "---------------------------------------------------------" 76 | cat < containing the files in git at $release_hash 113 | # the files in the tarball are prefixed with {version} (e.g. 4.0.1) 114 | mkdir -p ${distdir} 115 | (cd "${SOURCE_TOP_DIR}" && git archive ${release_hash} --prefix ${release}/ | gzip >${tarball}) 116 | 117 | echo "Running rat license checker on ${tarball}" 118 | ${SOURCE_DIR}/run-rat.sh ${tarball} 119 | 120 | echo "Signing tarball and creating checksums" 121 | gpg --armor --output ${tarball}.asc --detach-sig ${tarball} 122 | # create signing with relative path of tarball 123 | # so that they can be verified with a command such as 124 | # shasum --check apache-datafusion-ray-4.1.0-rc2.tar.gz.sha512 125 | (cd ${distdir} && shasum -a 256 ${tarname}) >${tarball}.sha256 126 | (cd ${distdir} && shasum -a 512 ${tarname}) >${tarball}.sha512 127 | 128 | echo "Uploading to datafusion dist/dev to ${url}" 129 | svn co --depth=empty https://dist.apache.org/repos/dist/dev/datafusion ${SOURCE_TOP_DIR}/dev/dist 130 | svn add ${distdir} 131 | svn ci -m "Apache DataFusion for Ray ${version} ${rc}" ${distdir} 132 | -------------------------------------------------------------------------------- /dev/release/generate-changelog.py: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env python 2 | 3 | # Licensed to the Apache Software Foundation (ASF) under one or more 4 | # contributor license agreements. See the NOTICE file distributed with 5 | # this work for additional information regarding copyright ownership. 6 | # The ASF licenses this file to You under the Apache License, Version 2.0 7 | # (the "License"); you may not use this file except in compliance with 8 | # the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | 18 | import argparse 19 | import sys 20 | from github import Github 21 | import os 22 | import re 23 | import subprocess 24 | 25 | 26 | def print_pulls(repo_name, title, pulls): 27 | if len(pulls) > 0: 28 | print("**{}:**".format(title)) 29 | print() 30 | for pull, commit in pulls: 31 | url = "https://github.com/{}/pull/{}".format( 32 | repo_name, pull.number 33 | ) 34 | print( 35 | "- {} [#{}]({}) ({})".format( 36 | pull.title, pull.number, url, commit.author.login 37 | ) 38 | ) 39 | print() 40 | 41 | 42 | def generate_changelog(repo, repo_name, tag1, tag2, version): 43 | 44 | # get a list of commits between two tags 45 | print( 46 | f"Fetching list of commits between {tag1} and {tag2}", file=sys.stderr 47 | ) 48 | comparison = repo.compare(tag1, tag2) 49 | 50 | # get the pull requests for these commits 51 | print("Fetching pull requests", file=sys.stderr) 52 | unique_pulls = [] 53 | all_pulls = [] 54 | for commit in comparison.commits: 55 | pulls = commit.get_pulls() 56 | for pull in pulls: 57 | # there can be multiple commits per PR if squash merge is not being used and 58 | # in this case we should get all the author names, but for now just pick one 59 | if pull.number not in unique_pulls: 60 | unique_pulls.append(pull.number) 61 | all_pulls.append((pull, commit)) 62 | 63 | # we split the pulls into categories 64 | breaking = [] 65 | bugs = [] 66 | docs = [] 67 | enhancements = [] 68 | performance = [] 69 | other = [] 70 | 71 | # categorize the pull requests based on GitHub labels 72 | print("Categorizing pull requests", file=sys.stderr) 73 | for pull, commit in all_pulls: 74 | 75 | # see if PR title uses Conventional Commits 76 | cc_type = "" 77 | cc_scope = "" 78 | cc_breaking = "" 79 | parts = re.findall(r"^([a-z]+)(\([a-z]+\))?(!)?:", pull.title) 80 | if len(parts) == 1: 81 | parts_tuple = parts[0] 82 | cc_type = parts_tuple[0] # fix, feat, docs, chore 83 | cc_scope = parts_tuple[1] # component within project 84 | cc_breaking = parts_tuple[2] == "!" 85 | 86 | labels = [label.name for label in pull.labels] 87 | if "api change" in labels or cc_breaking: 88 | breaking.append((pull, commit)) 89 | elif "bug" in labels or cc_type == "fix": 90 | bugs.append((pull, commit)) 91 | elif "performance" in labels or cc_type == "perf": 92 | performance.append((pull, commit)) 93 | elif "enhancement" in labels or cc_type == "feat": 94 | enhancements.append((pull, commit)) 95 | elif ( 96 | "documentation" in labels or cc_type == "docs" or cc_type == "doc" 97 | ): 98 | docs.append((pull, commit)) 99 | else: 100 | other.append((pull, commit)) 101 | 102 | # produce the changelog content 103 | print("Generating changelog content", file=sys.stderr) 104 | 105 | # ASF header 106 | print( 107 | """\n""" 125 | ) 126 | 127 | print(f"# Apache DataFusion for Ray {version} Changelog\n") 128 | 129 | # get the number of commits 130 | commit_count = subprocess.check_output( 131 | f"git log --pretty=oneline {tag1}..{tag2} | wc -l", 132 | shell=True, 133 | text=True, 134 | ).strip() 135 | 136 | # get number of contributors 137 | contributor_count = subprocess.check_output( 138 | f"git shortlog -sn {tag1}..{tag2} | wc -l", shell=True, text=True 139 | ).strip() 140 | 141 | print( 142 | f"This release consists of {commit_count} commits from {contributor_count} contributors. " 143 | f"See credits at the end of this changelog for more information.\n" 144 | ) 145 | 146 | print_pulls(repo_name, "Breaking changes", breaking) 147 | print_pulls(repo_name, "Performance related", performance) 148 | print_pulls(repo_name, "Implemented enhancements", enhancements) 149 | print_pulls(repo_name, "Fixed bugs", bugs) 150 | print_pulls(repo_name, "Documentation updates", docs) 151 | print_pulls(repo_name, "Other", other) 152 | 153 | # show code contributions 154 | credits = subprocess.check_output( 155 | f"git shortlog -sn {tag1}..{tag2}", shell=True, text=True 156 | ).rstrip() 157 | 158 | print("## Credits\n") 159 | print( 160 | "Thank you to everyone who contributed to this release. Here is a breakdown of commits (PRs merged) " 161 | "per contributor.\n" 162 | ) 163 | print("```") 164 | print(credits) 165 | print("```\n") 166 | 167 | print( 168 | "Thank you also to everyone who contributed in other ways such as filing issues, reviewing " 169 | "PRs, and providing feedback on this release.\n" 170 | ) 171 | 172 | 173 | def cli(args=None): 174 | """Process command line arguments.""" 175 | if not args: 176 | args = sys.argv[1:] 177 | 178 | parser = argparse.ArgumentParser() 179 | parser.add_argument("tag1", help="The previous commit or tag (e.g. 0.1.0)") 180 | parser.add_argument("tag2", help="The current commit or tag (e.g. HEAD)") 181 | parser.add_argument( 182 | "version", help="The version number to include in the changelog" 183 | ) 184 | args = parser.parse_args() 185 | 186 | token = os.getenv("GITHUB_TOKEN") 187 | project = "apache/datafusion-ray" 188 | 189 | g = Github(token) 190 | repo = g.get_repo(project) 191 | generate_changelog(repo, project, args.tag1, args.tag2, args.version) 192 | 193 | 194 | if __name__ == "__main__": 195 | cli() 196 | -------------------------------------------------------------------------------- /dev/release/rat_exclude_files.txt: -------------------------------------------------------------------------------- 1 | *.npmrc 2 | *.gitignore 3 | *.dockerignore 4 | .gitmodules 5 | *_generated.h 6 | *_generated.js 7 | *_generated.ts 8 | *.csv 9 | *.json 10 | *.snap 11 | .github/ISSUE_TEMPLATE/*.md 12 | .github/pull_request_template.md 13 | CHANGELOG.md 14 | dev/release/rat_exclude_files.txt 15 | MANIFEST.in 16 | __init__.pxd 17 | __init__.py 18 | *.html 19 | *.sgml 20 | *.css 21 | *.png 22 | *.ico 23 | *.svg 24 | *.devhelp2 25 | *.scss 26 | .gitattributes 27 | requirements.txt 28 | *requirements*.txt 29 | **/testdata/* 30 | ci/* 31 | **/*.svg 32 | **/*.csv 33 | **/*.json 34 | **/*.sql 35 | venv/* 36 | parquet/* 37 | testing/* 38 | target/* 39 | **/target/* 40 | Cargo.lock 41 | **/Cargo.lock 42 | .history 43 | *rat.txt 44 | */.git 45 | .github/* 46 | benchmarks/tpch/queries/q*.sql 47 | benchmarks/tpch/create_tables.sql 48 | .cargo/config.toml -------------------------------------------------------------------------------- /dev/release/release-tarball.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | # 3 | # Licensed to the Apache Software Foundation (ASF) under one 4 | # or more contributor license agreements. See the NOTICE file 5 | # distributed with this work for additional information 6 | # regarding copyright ownership. The ASF licenses this file 7 | # to you under the Apache License, Version 2.0 (the 8 | # "License"); you may not use this file except in compliance 9 | # with the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, 14 | # software distributed under the License is distributed on an 15 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | # KIND, either express or implied. See the License for the 17 | # specific language governing permissions and limitations 18 | # under the License. 19 | # 20 | 21 | # Adapted from https://github.com/apache/arrow-rs/tree/master/dev/release/release-tarball.sh 22 | 23 | # This script copies a tarball from the "dev" area of the 24 | # dist.apache.arrow repository to the "release" area 25 | # 26 | # This script should only be run after the release has been approved 27 | # by the arrow PMC committee. 28 | # 29 | # See release/README.md for full release instructions 30 | # 31 | # Based in part on post-01-upload.sh from apache/arrow 32 | 33 | set -e 34 | set -u 35 | 36 | if [ "$#" -ne 2 ]; then 37 | echo "Usage: $0 " 38 | echo "ex. $0 4.1.0 2" 39 | exit 40 | fi 41 | 42 | version=$1 43 | rc=$2 44 | 45 | tmp_dir=tmp-apache-datafusion-ray-dist 46 | 47 | echo "Recreate temporary directory: ${tmp_dir}" 48 | rm -rf ${tmp_dir} 49 | mkdir -p ${tmp_dir} 50 | 51 | echo "Clone dev dist repository" 52 | svn \ 53 | co \ 54 | https://dist.apache.org/repos/dist/dev/datafusion/apache-datafusion-ray-${version}-rc${rc} \ 55 | ${tmp_dir}/dev 56 | 57 | echo "Clone release dist repository" 58 | svn co https://dist.apache.org/repos/dist/release/datafusion ${tmp_dir}/release 59 | 60 | echo "Copy ${version}-rc${rc} to release working copy" 61 | release_version=datafusion-ray-${version} 62 | mkdir -p ${tmp_dir}/release/${release_version} 63 | cp -r ${tmp_dir}/dev/* ${tmp_dir}/release/${release_version}/ 64 | svn add ${tmp_dir}/release/${release_version} 65 | 66 | echo "Commit release" 67 | svn ci -m "Apache DataFusion for Ray ${version}" ${tmp_dir}/release 68 | 69 | echo "Clean up" 70 | rm -rf ${tmp_dir} 71 | 72 | echo "Success! The release is available here:" 73 | echo " https://dist.apache.org/repos/dist/release/datafusion/${release_version}" 74 | -------------------------------------------------------------------------------- /dev/release/run-rat.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | # 3 | # Licensed to the Apache Software Foundation (ASF) under one 4 | # or more contributor license agreements. See the NOTICE file 5 | # distributed with this work for additional information 6 | # regarding copyright ownership. The ASF licenses this file 7 | # to you under the Apache License, Version 2.0 (the 8 | # "License"); you may not use this file except in compliance 9 | # with the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, 14 | # software distributed under the License is distributed on an 15 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | # KIND, either express or implied. See the License for the 17 | # specific language governing permissions and limitations 18 | # under the License. 19 | # 20 | 21 | RAT_VERSION=0.13 22 | 23 | # download apache rat 24 | if [ ! -f apache-rat-${RAT_VERSION}.jar ]; then 25 | curl -s https://repo1.maven.org/maven2/org/apache/rat/apache-rat/${RAT_VERSION}/apache-rat-${RAT_VERSION}.jar > apache-rat-${RAT_VERSION}.jar 26 | fi 27 | 28 | RAT="java -jar apache-rat-${RAT_VERSION}.jar -x " 29 | 30 | RELEASE_DIR=$(cd "$(dirname "$BASH_SOURCE")"; pwd) 31 | 32 | # generate the rat report 33 | $RAT $1 > rat.txt 34 | python $RELEASE_DIR/check-rat-report.py $RELEASE_DIR/rat_exclude_files.txt rat.txt > filtered_rat.txt 35 | cat filtered_rat.txt 36 | UNAPPROVED=`cat filtered_rat.txt | grep "NOT APPROVED" | wc -l` 37 | 38 | if [ "0" -eq "${UNAPPROVED}" ]; then 39 | echo "No unapproved licenses" 40 | else 41 | echo "${UNAPPROVED} unapproved licences. Check rat report: rat.txt" 42 | exit 1 43 | fi 44 | -------------------------------------------------------------------------------- /dev/release/verify-release-candidate.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # 3 | # Licensed to the Apache Software Foundation (ASF) under one 4 | # or more contributor license agreements. See the NOTICE file 5 | # distributed with this work for additional information 6 | # regarding copyright ownership. The ASF licenses this file 7 | # to you under the Apache License, Version 2.0 (the 8 | # "License"); you may not use this file except in compliance 9 | # with the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, 14 | # software distributed under the License is distributed on an 15 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | # KIND, either express or implied. See the License for the 17 | # specific language governing permissions and limitations 18 | # under the License. 19 | # 20 | 21 | case $# in 22 | 2) VERSION="$1" 23 | RC_NUMBER="$2" 24 | ;; 25 | *) echo "Usage: $0 X.Y.Z RC_NUMBER" 26 | exit 1 27 | ;; 28 | esac 29 | 30 | set -e 31 | set -x 32 | set -o pipefail 33 | 34 | SOURCE_DIR="$(cd "$(dirname "${BASH_SOURCE[0]:-$0}")" && pwd)" 35 | DATAFUSION_RAY_DIR="$(dirname $(dirname ${SOURCE_DIR}))" 36 | DATAFUSION_RAY_DIST_URL='https://dist.apache.org/repos/dist/dev/datafusion' 37 | 38 | download_dist_file() { 39 | curl \ 40 | --silent \ 41 | --show-error \ 42 | --fail \ 43 | --location \ 44 | --remote-name $DATAFUSION_RAY_DIST_URL/$1 45 | } 46 | 47 | download_rc_file() { 48 | download_dist_file apache-datafusion-ray-${VERSION}-rc${RC_NUMBER}/$1 49 | } 50 | 51 | import_gpg_keys() { 52 | download_dist_file KEYS 53 | gpg --import KEYS 54 | } 55 | 56 | if type shasum >/dev/null 2>&1; then 57 | sha256_verify="shasum -a 256 -c" 58 | sha512_verify="shasum -a 512 -c" 59 | else 60 | sha256_verify="sha256sum -c" 61 | sha512_verify="sha512sum -c" 62 | fi 63 | 64 | fetch_archive() { 65 | local dist_name=$1 66 | download_rc_file ${dist_name}.tar.gz 67 | download_rc_file ${dist_name}.tar.gz.asc 68 | download_rc_file ${dist_name}.tar.gz.sha256 69 | download_rc_file ${dist_name}.tar.gz.sha512 70 | verify_dir_artifact_signatures 71 | } 72 | 73 | verify_dir_artifact_signatures() { 74 | # verify the signature and the checksums of each artifact 75 | find . -name '*.asc' | while read sigfile; do 76 | artifact=${sigfile/.asc/} 77 | gpg --verify $sigfile $artifact || exit 1 78 | 79 | # go into the directory because the checksum files contain only the 80 | # basename of the artifact 81 | pushd $(dirname $artifact) 82 | base_artifact=$(basename $artifact) 83 | ${sha256_verify} $base_artifact.sha256 || exit 1 84 | ${sha512_verify} $base_artifact.sha512 || exit 1 85 | popd 86 | done 87 | } 88 | 89 | setup_tempdir() { 90 | cleanup() { 91 | if [ "${TEST_SUCCESS}" = "yes" ]; then 92 | rm -fr "${DATAFUSION_RAY_TMPDIR}" 93 | else 94 | echo "Failed to verify release candidate. See ${DATAFUSION_RAY_TMPDIR} for details." 95 | fi 96 | } 97 | 98 | if [ -z "${DATAFUSION_RAY_TMPDIR}" ]; then 99 | # clean up automatically if DATAFUSION_RAY_TMPDIR is not defined 100 | DATAFUSION_RAY_TMPDIR=$(mktemp -d -t "$1.XXXXX") 101 | trap cleanup EXIT 102 | else 103 | # don't clean up automatically 104 | mkdir -p "${DATAFUSION_RAY_TMPDIR}" 105 | fi 106 | } 107 | 108 | test_source_distribution() { 109 | # install rust toolchain in a similar fashion like test-miniconda 110 | export RUSTUP_HOME=$PWD/test-rustup 111 | export CARGO_HOME=$PWD/test-rustup 112 | 113 | curl https://sh.rustup.rs -sSf | sh -s -- -y --no-modify-path 114 | 115 | export PATH=$RUSTUP_HOME/bin:$PATH 116 | source $RUSTUP_HOME/env 117 | 118 | # build and test rust 119 | 120 | # raises on any formatting errors 121 | rustup component add rustfmt --toolchain stable 122 | cargo fmt --all -- --check 123 | 124 | python3 -m venv venv 125 | source venv/bin/activate 126 | python3 -m pip install -U pip 127 | python3 -m pip install -r requirements-in.txt 128 | maturin develop 129 | 130 | #TODO: we should really run tests here as well 131 | #python3 -m pytest 132 | 133 | if ( find -iname 'Cargo.toml' | xargs grep SNAPSHOT ); then 134 | echo "Cargo.toml version should not contain SNAPSHOT for releases" 135 | exit 1 136 | fi 137 | } 138 | 139 | TEST_SUCCESS=no 140 | 141 | setup_tempdir "datafusion-ray-${VERSION}" 142 | echo "Working in sandbox ${DATAFUSION_RAY_TMPDIR}" 143 | cd ${DATAFUSION_RAY_TMPDIR} 144 | 145 | dist_name="apache-datafusion-ray-${VERSION}" 146 | import_gpg_keys 147 | fetch_archive ${dist_name} 148 | tar xf ${dist_name}.tar.gz 149 | pushd ${dist_name} 150 | test_source_distribution 151 | popd 152 | 153 | TEST_SUCCESS=yes 154 | echo 'Release candidate looks good!' 155 | exit 0 156 | -------------------------------------------------------------------------------- /docs/benchmarks/df-ray-0.1.0rc1/i4g.4xlarge: -------------------------------------------------------------------------------- 1 | ================================================================================================= 2 | Spark and DataFusion for Ray TPCH 100 Benchmarks 3 | Machine: i4g.4xlarge 4 | Machine On Demand Cost: 1.23552 $/hr 5 | CPU(s): Neoverse-N1 16x 6 | MEM: 126.8G 7 | HD Throughput: 888.24 MB/s (from hdparm) 8 | Data Location: /data/sf100 9 | 10 | df-ray duration: 870.95s +1.04x faster 11 | df-ray cost: $0.2989 +1.04x cheaper 12 | 13 | spark duration: 906.69s 14 | spark cost: $0.3112 15 | 16 | DataFusion for Ray Settings: 17 | concurrency: 16 18 | batch_size : 8192 19 | partitions_per_processor: 4 20 | Ray Workers: 4 21 | Ray Worker Mem (GB): 24 22 | Ray Worker CPU: 3 23 | Ray Head Mem (GB): 4 24 | Ray Head CPU: 2 25 | 26 | Spark Settings: 27 | Executors: 4 28 | Executor Mem (GB): 20 29 | Executor Overhead Mem (GB): 4 30 | Executor CPU: 3 31 | Driver Mem(GB): 4 32 | Driver CPU: 2 33 | ================================================================================================= 34 | DataFrame() 35 | +------------+--------------------+--------------------+-----------------------+---------------+ 36 | | tpch_query | spark | df_ray | change(=df_ray/spark) | change_text | 37 | +------------+--------------------+--------------------+-----------------------+---------------+ 38 | | 1 | 107.42937994003296 | 40.58694672584534 | 0.3778011820276814 | +2.65x faster | 39 | | 2 | 14.26254153251648 | 14.748653411865234 | 1.0340831175313665 | 1.03x slower | 40 | | 3 | 34.457966566085815 | 23.512167930603027 | 0.6823434541765488 | +1.47x faster | 41 | | 4 | 24.979174375534058 | 12.797027349472046 | 0.5123078592223664 | +1.95x faster | 42 | | 5 | 57.75347018241882 | 46.364267349243164 | 0.8027962164489514 | +1.25x faster | 43 | | 6 | 6.254350423812866 | 20.64115333557129 | 3.3002873099310186 | 3.30x slower | 44 | | 7 | 21.208117723464966 | 67.15978837013245 | 3.1667019792060986 | 3.17x slower | 45 | | 8 | 56.11615228652954 | 52.393213748931885 | 0.9336565607957527 | +1.07x faster | 46 | | 9 | 108.6072256565094 | 72.31336116790771 | 0.6658245869995078 | +1.50x faster | 47 | | 10 | 28.727190732955933 | 40.29949235916138 | 1.4028344342396717 | 1.40x slower | 48 | | 11 | 4.468253135681152 | 13.295583248138428 | 2.9755662547331516 | 2.98x slower | 49 | | 12 | 18.66551184654236 | 15.770261764526367 | 0.8448877209572845 | +1.18x faster | 50 | | 13 | 38.409486532211304 | 19.281179428100586 | 0.5019900334239259 | +1.99x faster | 51 | | 14 | 8.859760522842407 | 12.571092128753662 | 1.4188975081598003 | 1.42x slower | 52 | | 15 | 23.616262197494507 | 26.80106806755066 | 1.1348564748910195 | 1.13x slower | 53 | | 16 | 11.123925685882568 | 12.223425388336182 | 1.098840978760672 | 1.10x slower | 54 | | 17 | 119.728675365448 | 82.61100387573242 | 0.6899851152915435 | +1.45x faster | 55 | | 18 | 88.93902206420898 | 116.58043599128723 | 1.3107906213216816 | 1.31x slower | 56 | | 19 | 13.113885879516602 | 34.65487003326416 | 2.642608785195681 | 2.64x slower | 57 | | 20 | 16.772423267364502 | 24.642067432403564 | 1.4692013813145106 | 1.47x slower | 58 | | 21 | 88.84716391563416 | 112.72514128684998 | 1.2687533998709224 | 1.27x slower | 59 | | 22 | 14.35309910774231 | 8.979067087173462 | 0.6255838561255387 | +1.60x faster | 60 | | total | 906.6930389404297 | 870.9512674808502 | 0.9605800751472101 | +1.04x faster | 61 | +------------+--------------------+--------------------+-----------------------+---------------+ 62 | -------------------------------------------------------------------------------- /docs/benchmarks/df-ray-0.1.0rc1/i4g.4xlarge_s3: -------------------------------------------------------------------------------- 1 | ================================================================================ 2 | Spark and DataFusionRay TPCH 100 Benchmarks 3 | Machine: 1x i4g.4xlarge 4 | Per Machine On Demand Cost: 1.23552 $/hr 5 | CPU TOTAL: 16x Neoverse-N1x 6 | MEM TOTAL: 126.8GB 7 | HD Throughput: 921.58 MB/s (from hdparm) 8 | Data Location: s3://rob-tandy-tmp//sf100 9 | 10 | df-ray duration: 1424.35s +2.44x faster 11 | df-ray cost: $0.4888 +2.44x cheaper 12 | 13 | spark duration: 3473.58s 14 | spark cost: $1.1921 15 | 16 | DataFusionRay Settings: 17 | concurrency: 16 18 | batch_size : 8192 19 | partitions_per_processor: 4 20 | Ray Workers: 4 21 | Ray Worker Mem (GB): 24 22 | Ray Worker CPU: 3 23 | Ray Head Mem (GB): 4 24 | Ray Head CPU: 2 25 | 26 | Spark Settings: 27 | Executors: 4 28 | Executor Mem (GB): 20 29 | Executor Overhead Mem (GB): 4 30 | Executor CPU: 3 31 | Driver Mem(GB): 4 32 | Driver CPU: 2 33 | ================================================================================ 34 | DataFrame() 35 | +------------+------------+------------+-----------------------+---------------+ 36 | | tpch_query | spark | df_ray | change(=df_ray/spark) | change_text | 37 | +------------+------------+------------+-----------------------+---------------+ 38 | | 1 | 223.7065 | 53.2348 | 0.2380 | +4.20x faster | 39 | | 2 | 39.5393 | 25.1823 | 0.6369 | +1.57x faster | 40 | | 3 | 186.6593 | 58.9956 | 0.3161 | +3.16x faster | 41 | | 4 | 126.7456 | 27.6134 | 0.2179 | +4.59x faster | 42 | | 5 | 218.8699 | 102.4856 | 0.4682 | +2.14x faster | 43 | | 6 | 96.1340 | 33.1685 | 0.3450 | +2.90x faster | 44 | | 7 | 210.9092 | 98.8647 | 0.4688 | +2.13x faster | 45 | | 8 | 187.0856 | 70.3633 | 0.3761 | +2.66x faster | 46 | | 9 | 203.7582 | 105.3517 | 0.5170 | +1.93x faster | 47 | | 10 | 110.8606 | 72.4830 | 0.6538 | +1.53x faster | 48 | | 11 | 20.4039 | 19.8943 | 0.9750 | +1.03x faster | 49 | | 12 | 96.4592 | 49.0558 | 0.5086 | +1.97x faster | 50 | | 13 | 61.5240 | 20.0987 | 0.3267 | +3.06x faster | 51 | | 14 | 143.0664 | 26.4152 | 0.1846 | +5.42x faster | 52 | | 15 | 307.8785 | 71.2565 | 0.2314 | +4.32x faster | 53 | | 16 | 20.2879 | 14.6983 | 0.7245 | +1.38x faster | 54 | | 17 | 301.2155 | 148.0474 | 0.4915 | +2.03x faster | 55 | | 18 | 242.4097 | 144.3459 | 0.5955 | +1.68x faster | 56 | | 19 | 89.4152 | 92.0230 | 1.0292 | 1.03x slower | 57 | | 20 | 123.9332 | 39.6039 | 0.3196 | +3.13x faster | 58 | | 21 | 428.6912 | 140.1514 | 0.3269 | +3.06x faster | 59 | | 22 | 34.0295 | 11.0184 | 0.3238 | +3.09x faster | 60 | | total | 3473.5824 | 1424.3519 | 0.4101 | +2.44x faster | 61 | +------------+------------+------------+-----------------------+---------------+ 62 | -------------------------------------------------------------------------------- /docs/benchmarks/df-ray-0.1.0rc1/m7g.12x: -------------------------------------------------------------------------------- 1 | ================================================================================= 2 | Spark and DataFusionRay TPCH 100 Benchmarks 3 | Machine: m7g.12xlarge 4 | Machine On Demand Cost: 1.9584 $/hr 5 | CPU(s): Neoverse-V1 48x 6 | MEM: 189G 7 | HD Throughput: 166.46 MB/s (from hdparm) 8 | Data Location: /data/sf100 9 | 10 | df-ray duration: 204.97s +4.75x faster 11 | df-ray cost: $0.1115 +4.75x cheaper 12 | 13 | spark duration: 974.60s 14 | spark cost: $0.5302 15 | 16 | DataFusionRay Settings: 17 | concurrency: 16 18 | batch_size : 8192 19 | partitions_per_processor: 1 20 | Ray Workers: 4 21 | Ray Worker Mem (GB): 40 22 | Ray Worker CPU: 10 23 | Ray Head Mem (GB): 10 24 | Ray Head CPU: 4 25 | 26 | Spark Settings: 27 | Executors: 4 28 | Executor Mem (GB): 32 29 | Executor Overhead Mem (GB): 8 30 | Executor CPU: 10 31 | Driver Mem(GB): 10 32 | Driver CPU: 4 33 | ================================================================================= 34 | DataFrame() 35 | +------------+------------+------------+-----------------------+----------------+ 36 | | tpch_query | spark | df_ray | change(=df_ray/spark) | change_text | 37 | +------------+------------+------------+-----------------------+----------------+ 38 | | 1 | 41.0527 | 7.3332 | 0.1786 | +5.60x faster | 39 | | 2 | 5.2886 | 3.3632 | 0.6359 | +1.57x faster | 40 | | 3 | 8.8360 | 5.8260 | 0.6593 | +1.52x faster | 41 | | 4 | 6.7364 | 2.8624 | 0.4249 | +2.35x faster | 42 | | 5 | 113.1327 | 9.7685 | 0.0863 | +11.58x faster | 43 | | 6 | 4.0279 | 5.0924 | 1.2643 | 1.26x slower | 44 | | 7 | 10.3482 | 15.0313 | 1.4526 | 1.45x slower | 45 | | 8 | 197.1830 | 10.5564 | 0.0535 | +18.68x faster | 46 | | 9 | 244.0854 | 17.0069 | 0.0697 | +14.35x faster | 47 | | 10 | 25.6980 | 10.1537 | 0.3951 | +2.53x faster | 48 | | 11 | 2.0117 | 3.2634 | 1.6222 | 1.62x slower | 49 | | 12 | 20.2309 | 4.5518 | 0.2250 | +4.44x faster | 50 | | 13 | 8.1399 | 3.5019 | 0.4302 | +2.32x faster | 51 | | 14 | 2.5995 | 3.3954 | 1.3062 | 1.31x slower | 52 | | 15 | 6.7179 | 6.2090 | 0.9243 | +1.08x faster | 53 | | 16 | 4.2510 | 3.1190 | 0.7337 | +1.36x faster | 54 | | 17 | 119.6635 | 21.6696 | 0.1811 | +5.52x faster | 55 | | 18 | 61.9433 | 32.2979 | 0.5214 | +1.92x faster | 56 | | 19 | 4.1546 | 5.9170 | 1.4242 | 1.42x slower | 57 | | 20 | 5.0565 | 5.5918 | 1.1059 | 1.11x slower | 58 | | 21 | 78.7263 | 26.4413 | 0.3359 | +2.98x faster | 59 | | 22 | 4.7114 | 2.0223 | 0.4292 | +2.33x faster | 60 | | total | 974.5953 | 204.9744 | 0.2103 | +4.75x faster | 61 | +------------+------------+------------+-----------------------+----------------+ 62 | 63 | 64 | ========================================================================================== 65 | Spark and DataFusion for Ray TPCH 100 Benchmarks 66 | Machine: m7g.12xlarge 67 | CPU(s): 48x 68 | MEM: 189G 69 | HD Throughput: 167.53 MB/s (from hdparm) 70 | 71 | DataFusion for Ray Settings: 72 | concurrency: 16 73 | batch_size : 8192 74 | partitions_per_processor: 2 75 | Ray Workers: 4 76 | Ray Worker Mem (GB): 40 77 | Ray Worker CPU: 10 78 | Ray Head Mem (GB): 10 79 | Ray Head CPU: 4 80 | 81 | Spark Settings: 82 | Executors: 4 83 | Executor Mem (GB): 32 84 | Executor Overhead Mem (GB): 8 85 | Executor CPU: 10 86 | Driver Mem(GB): 10 87 | Driver CPU: 4 88 | ========================================================================================== 89 | DataFrame() 90 | +------------+--------------------+--------------------+---------------------+----------------+ 91 | | tpch_query | spark | df_ray | change | change_text | 92 | +------------+--------------------+--------------------+---------------------+----------------+ 93 | | 1 | 40.66942858695984 | 7.127374172210693 | 0.1752513969300272 | +5.71x faster | 94 | | 2 | 5.27202296257019 | 4.556609869003296 | 0.8643000801312671 | +1.16x faster | 95 | | 3 | 9.467089414596558 | 6.217605829238892 | 0.6567600195739628 | +1.52x faster | 96 | | 4 | 6.713418006896973 | 2.6009602546691895 | 0.3874271275819732 | +2.58x faster | 97 | | 5 | 138.02316689491272 | 14.755179166793823 | 0.10690364160407967 | +9.35x faster | 98 | | 6 | 1.7651944160461426 | 4.644023895263672 | 2.630885217542109 | 2.63x slower | 99 | | 7 | 7.105847120285034 | 12.337952613830566 | 1.7363098874741423 | 1.74x slower | 100 | | 8 | 234.60386061668396 | 8.625483512878418 | 0.03676616186198009 | +27.20x faster | 101 | | 9 | 308.04562306404114 | 15.377676725387573 | 0.04992012732539502 | +20.03x faster | 102 | | 10 | 99.11399936676025 | 12.474991083145142 | 0.12586507620364337 | +7.95x faster | 103 | | 11 | 2.5937740802764893 | 2.8647923469543457 | 1.104488000222813 | 1.10x slower | 104 | | 12 | 22.38423728942871 | 4.3449485301971436 | 0.1941075085122114 | +5.15x faster | 105 | | 13 | 31.56517219543457 | 3.571704864501953 | 0.11315334642839511 | +8.84x faster | 106 | | 14 | 16.334258317947388 | 3.2847304344177246 | 0.20109455663552245 | +4.97x faster | 107 | | 15 | 11.267093658447266 | 5.8551459312438965 | 0.5196678139667477 | +1.92x faster | 108 | | 16 | 6.11682653427124 | 2.544823408126831 | 0.4160365499771397 | +2.40x faster | 109 | | 17 | 139.6458387374878 | 17.490234375 | 0.12524708600790382 | +7.98x faster | 110 | | 18 | 68.82194375991821 | 29.644872903823853 | 0.4307473936981271 | +2.32x faster | 111 | | 19 | 11.364177465438843 | 5.922780275344849 | 0.5211798472311284 | +1.92x faster | 112 | | 20 | 15.337732791900635 | 4.3357462882995605 | 0.28268495397110643 | +3.54x faster | 113 | | 21 | 114.92279720306396 | 24.284620761871338 | 0.21131247544350482 | +4.73x faster | 114 | | 22 | 6.929309368133545 | 1.8829975128173828 | 0.27174389434492524 | +3.68x faster | 115 | | total | 1298.0628118515015 | 194.74525475502014 | 0.15002760496408013 | +6.67x faster | 116 | +------------+--------------------+--------------------+---------------------+----------------+ 117 | 118 | -------------------------------------------------------------------------------- /docs/benchmarks/df-ray-0.1.0rc1/m7g.12xlarge: -------------------------------------------------------------------------------- 1 | ================================================================================= 2 | Spark and DataFusionRay TPCH 100 Benchmarks 3 | Machine: m7g.12xlarge 4 | Machine On Demand Cost: 1.9584 $/hr 5 | CPU(s): Neoverse-V1 48x 6 | MEM: 189G 7 | HD Throughput: 166.46 MB/s (from hdparm) 8 | Data Location: /data/sf100 9 | 10 | df-ray duration: 204.97s +4.75x faster 11 | df-ray cost: $0.1115 +4.75x cheaper 12 | 13 | spark duration: 974.60s 14 | spark cost: $0.5302 15 | 16 | DataFusionRay Settings: 17 | concurrency: 16 18 | batch_size : 8192 19 | partitions_per_processor: 1 20 | Ray Workers: 4 21 | Ray Worker Mem (GB): 40 22 | Ray Worker CPU: 10 23 | Ray Head Mem (GB): 10 24 | Ray Head CPU: 4 25 | 26 | Spark Settings: 27 | Executors: 4 28 | Executor Mem (GB): 32 29 | Executor Overhead Mem (GB): 8 30 | Executor CPU: 10 31 | Driver Mem(GB): 10 32 | Driver CPU: 4 33 | ================================================================================= 34 | DataFrame() 35 | +------------+------------+------------+-----------------------+----------------+ 36 | | tpch_query | spark | df_ray | change(=df_ray/spark) | change_text | 37 | +------------+------------+------------+-----------------------+----------------+ 38 | | 1 | 41.0527 | 7.3332 | 0.1786 | +5.60x faster | 39 | | 2 | 5.2886 | 3.3632 | 0.6359 | +1.57x faster | 40 | | 3 | 8.8360 | 5.8260 | 0.6593 | +1.52x faster | 41 | | 4 | 6.7364 | 2.8624 | 0.4249 | +2.35x faster | 42 | | 5 | 113.1327 | 9.7685 | 0.0863 | +11.58x faster | 43 | | 6 | 4.0279 | 5.0924 | 1.2643 | 1.26x slower | 44 | | 7 | 10.3482 | 15.0313 | 1.4526 | 1.45x slower | 45 | | 8 | 197.1830 | 10.5564 | 0.0535 | +18.68x faster | 46 | | 9 | 244.0854 | 17.0069 | 0.0697 | +14.35x faster | 47 | | 10 | 25.6980 | 10.1537 | 0.3951 | +2.53x faster | 48 | | 11 | 2.0117 | 3.2634 | 1.6222 | 1.62x slower | 49 | | 12 | 20.2309 | 4.5518 | 0.2250 | +4.44x faster | 50 | | 13 | 8.1399 | 3.5019 | 0.4302 | +2.32x faster | 51 | | 14 | 2.5995 | 3.3954 | 1.3062 | 1.31x slower | 52 | | 15 | 6.7179 | 6.2090 | 0.9243 | +1.08x faster | 53 | | 16 | 4.2510 | 3.1190 | 0.7337 | +1.36x faster | 54 | | 17 | 119.6635 | 21.6696 | 0.1811 | +5.52x faster | 55 | | 18 | 61.9433 | 32.2979 | 0.5214 | +1.92x faster | 56 | | 19 | 4.1546 | 5.9170 | 1.4242 | 1.42x slower | 57 | | 20 | 5.0565 | 5.5918 | 1.1059 | 1.11x slower | 58 | | 21 | 78.7263 | 26.4413 | 0.3359 | +2.98x faster | 59 | | 22 | 4.7114 | 2.0223 | 0.4292 | +2.33x faster | 60 | | total | 974.5953 | 204.9744 | 0.2103 | +4.75x faster | 61 | +------------+------------+------------+-----------------------+----------------+ 62 | 63 | 64 | ========================================================================================== 65 | Spark and DataFusion for Ray TPCH 100 Benchmarks 66 | Machine: m7g.12xlarge 67 | CPU(s): 48x 68 | MEM: 189G 69 | HD Throughput: 167.53 MB/s (from hdparm) 70 | 71 | DataFusion for Ray Settings: 72 | concurrency: 16 73 | batch_size : 8192 74 | partitions_per_processor: 2 75 | Ray Workers: 4 76 | Ray Worker Mem (GB): 40 77 | Ray Worker CPU: 10 78 | Ray Head Mem (GB): 10 79 | Ray Head CPU: 4 80 | 81 | Spark Settings: 82 | Executors: 4 83 | Executor Mem (GB): 32 84 | Executor Overhead Mem (GB): 8 85 | Executor CPU: 10 86 | Driver Mem(GB): 10 87 | Driver CPU: 4 88 | ========================================================================================== 89 | DataFrame() 90 | +------------+--------------------+--------------------+---------------------+----------------+ 91 | | tpch_query | spark | df_ray | change | change_text | 92 | +------------+--------------------+--------------------+---------------------+----------------+ 93 | | 1 | 40.66942858695984 | 7.127374172210693 | 0.1752513969300272 | +5.71x faster | 94 | | 2 | 5.27202296257019 | 4.556609869003296 | 0.8643000801312671 | +1.16x faster | 95 | | 3 | 9.467089414596558 | 6.217605829238892 | 0.6567600195739628 | +1.52x faster | 96 | | 4 | 6.713418006896973 | 2.6009602546691895 | 0.3874271275819732 | +2.58x faster | 97 | | 5 | 138.02316689491272 | 14.755179166793823 | 0.10690364160407967 | +9.35x faster | 98 | | 6 | 1.7651944160461426 | 4.644023895263672 | 2.630885217542109 | 2.63x slower | 99 | | 7 | 7.105847120285034 | 12.337952613830566 | 1.7363098874741423 | 1.74x slower | 100 | | 8 | 234.60386061668396 | 8.625483512878418 | 0.03676616186198009 | +27.20x faster | 101 | | 9 | 308.04562306404114 | 15.377676725387573 | 0.04992012732539502 | +20.03x faster | 102 | | 10 | 99.11399936676025 | 12.474991083145142 | 0.12586507620364337 | +7.95x faster | 103 | | 11 | 2.5937740802764893 | 2.8647923469543457 | 1.104488000222813 | 1.10x slower | 104 | | 12 | 22.38423728942871 | 4.3449485301971436 | 0.1941075085122114 | +5.15x faster | 105 | | 13 | 31.56517219543457 | 3.571704864501953 | 0.11315334642839511 | +8.84x faster | 106 | | 14 | 16.334258317947388 | 3.2847304344177246 | 0.20109455663552245 | +4.97x faster | 107 | | 15 | 11.267093658447266 | 5.8551459312438965 | 0.5196678139667477 | +1.92x faster | 108 | | 16 | 6.11682653427124 | 2.544823408126831 | 0.4160365499771397 | +2.40x faster | 109 | | 17 | 139.6458387374878 | 17.490234375 | 0.12524708600790382 | +7.98x faster | 110 | | 18 | 68.82194375991821 | 29.644872903823853 | 0.4307473936981271 | +2.32x faster | 111 | | 19 | 11.364177465438843 | 5.922780275344849 | 0.5211798472311284 | +1.92x faster | 112 | | 20 | 15.337732791900635 | 4.3357462882995605 | 0.28268495397110643 | +3.54x faster | 113 | | 21 | 114.92279720306396 | 24.284620761871338 | 0.21131247544350482 | +4.73x faster | 114 | | 22 | 6.929309368133545 | 1.8829975128173828 | 0.27174389434492524 | +3.68x faster | 115 | | total | 1298.0628118515015 | 194.74525475502014 | 0.15002760496408013 | +6.67x faster | 116 | +------------+--------------------+--------------------+---------------------+----------------+ 117 | 118 | -------------------------------------------------------------------------------- /docs/benchmarks/df-ray-0.1.0rc1/m7g.12xlarge_s3: -------------------------------------------------------------------------------- 1 | ================================================================================= 2 | Spark and DataFusionRay TPCH 100 Benchmarks 3 | Machine: m7g.12xlarge 4 | Machine On Demand Cost: 1.9584 $/hr 5 | CPU(s): Neoverse-V1 48x 6 | MEM: 189G 7 | HD Throughput: 167.24 MB/s (from hdparm) 8 | Data Location: s3://rob-tandy-tmp/sf100 9 | 10 | df-ray duration: 480.91s +3.06x faster 11 | df-ray cost: $0.2616 +3.06x cheaper 12 | 13 | spark duration: 1469.31s 14 | spark cost: $0.7993 15 | 16 | DataFusionRay Settings: 17 | concurrency: 32 18 | batch_size : 8192 19 | partitions_per_processor: 1 20 | Ray Workers: 4 21 | Ray Worker Mem (GB): 40 22 | Ray Worker CPU: 10 23 | Ray Head Mem (GB): 10 24 | Ray Head CPU: 4 25 | 26 | Spark Settings: 27 | Executors: 4 28 | Executor Mem (GB): 32 29 | Executor Overhead Mem (GB): 8 30 | Executor CPU: 10 31 | Driver Mem(GB): 10 32 | Driver CPU: 4 33 | ================================================================================= 34 | DataFrame() 35 | +------------+------------+------------+-----------------------+----------------+ 36 | | tpch_query | spark | df_ray | change(=df_ray/spark) | change_text | 37 | +------------+------------+------------+-----------------------+----------------+ 38 | | 1 | 58.4442 | 20.6952 | 0.3541 | +2.82x faster | 39 | | 2 | 13.5071 | 14.9597 | 1.1075 | 1.11x slower | 40 | | 3 | 61.9178 | 34.7799 | 0.5617 | +1.78x faster | 41 | | 4 | 43.8635 | 15.1339 | 0.3450 | +2.90x faster | 42 | | 5 | 96.1693 | 23.3548 | 0.2429 | +4.12x faster | 43 | | 6 | 33.6715 | 10.1410 | 0.3012 | +3.32x faster | 44 | | 7 | 67.6622 | 25.9438 | 0.3834 | +2.61x faster | 45 | | 8 | 202.4597 | 25.1409 | 0.1242 | +8.05x faster | 46 | | 9 | 243.0195 | 29.2583 | 0.1204 | +8.31x faster | 47 | | 10 | 40.6520 | 27.6224 | 0.6795 | +1.47x faster | 48 | | 11 | 7.3265 | 8.2168 | 1.1215 | 1.12x slower | 49 | | 12 | 34.8475 | 21.6464 | 0.6212 | +1.61x faster | 50 | | 13 | 14.5846 | 7.1863 | 0.4927 | +2.03x faster | 51 | | 14 | 50.6580 | 11.1040 | 0.2192 | +4.56x faster | 52 | | 15 | 102.5048 | 20.0342 | 0.1954 | +5.12x faster | 53 | | 16 | 7.2824 | 5.2953 | 0.7271 | +1.38x faster | 54 | | 17 | 111.3701 | 34.5333 | 0.3101 | +3.23x faster | 55 | | 18 | 74.6563 | 37.2044 | 0.4983 | +2.01x faster | 56 | | 19 | 30.9803 | 32.1251 | 1.0370 | 1.04x slower | 57 | | 20 | 39.0791 | 17.3910 | 0.4450 | +2.25x faster | 58 | | 21 | 124.4177 | 52.3758 | 0.4210 | +2.38x faster | 59 | | 22 | 10.2409 | 6.7682 | 0.6609 | +1.51x faster | 60 | | total | 1469.3147 | 480.9106 | 0.3273 | +3.06x faster | 61 | +------------+------------+------------+-----------------------+----------------+ 62 | -------------------------------------------------------------------------------- /docs/benchmarks/df-ray-0.1.0rc1/m7gd.12x: -------------------------------------------------------------------------------- 1 | ========================================================================================== 2 | Spark and DataFusion for Ray TPCH 100 Benchmarks 3 | Machine: m7gd.12xlarge 4 | CPU(s): 48x 5 | MEM: 189G 6 | HD Throughput: 2162.93 (from hdparm) 7 | 8 | DataFusion for Ray Settings: 9 | concurrency: 16 10 | batch_size : 8192 11 | partitions_per_processor: 2 12 | Ray Workers: 4 13 | Ray Worker Mem (GB): 40 14 | Ray Worker CPU: 10 15 | Ray Head Mem (GB): 10 16 | Ray Head CPU: 4 17 | 18 | Spark Settings: 19 | Executors: 4 20 | Executor Mem (GB): 32 21 | Executor Overhead Mem (GB): 8 22 | Executor CPU: 10 23 | Driver Mem(GB): 10 24 | Driver CPU: 4 25 | ========================================================================================== 26 | DataFrame() 27 | +------------+--------------------+--------------------+---------------------+---------------+ 28 | | tpch_query | spark | df_ray | change | change_text | 29 | +------------+--------------------+--------------------+---------------------+---------------+ 30 | | 1 | 40.10749578475952 | 7.07772159576416 | 0.17646879859434228 | +5.67x faster | 31 | | 2 | 5.265030860900879 | 4.421863079071045 | 0.8398551111843696 | +1.19x faster | 32 | | 3 | 8.993131875991821 | 4.821190357208252 | 0.5360969263754447 | +1.87x faster | 33 | | 4 | 8.75968074798584 | 2.8997082710266113 | 0.3310289900340668 | +3.02x faster | 34 | | 5 | 19.33255624771118 | 10.249737024307251 | 0.5301801217063955 | +1.89x faster | 35 | | 6 | 1.754713773727417 | 4.5369789600372314 | 2.5855948861674696 | 2.59x slower | 36 | | 7 | 6.20657753944397 | 13.625978946685791 | 2.1954094442694267 | 2.20x slower | 37 | | 8 | 24.33410143852234 | 9.368516683578491 | 0.3849953821901789 | +2.60x faster | 38 | | 9 | 38.12766456604004 | 13.534857511520386 | 0.35498784584817605 | +2.82x faster | 39 | | 10 | 7.784897565841675 | 8.692375421524048 | 1.1165690168698141 | 1.12x slower | 40 | | 11 | 2.0291600227355957 | 3.2606608867645264 | 1.6069017969162889 | 1.61x slower | 41 | | 12 | 6.192300081253052 | 4.058101415634155 | 0.6553463757223104 | +1.53x faster | 42 | | 13 | 8.62465763092041 | 3.507314682006836 | 0.40666132292981705 | +2.46x faster | 43 | | 14 | 2.430983304977417 | 3.096514940261841 | 1.2737705495228016 | 1.27x slower | 44 | | 15 | 6.798931360244751 | 5.77943229675293 | 0.8500501020714643 | +1.18x faster | 45 | | 16 | 4.050042390823364 | 3.0604259967803955 | 0.7556528306258588 | +1.32x faster | 46 | | 17 | 33.42715358734131 | 19.431111812591553 | 0.5812972307624187 | +1.72x faster | 47 | | 18 | 23.402374744415283 | 28.70749068260193 | 1.2266913506054615 | 1.23x slower | 48 | | 19 | 3.6461265087127686 | 5.904490232467651 | 1.6193870998053157 | 1.62x slower | 49 | | 20 | 5.166107892990112 | 5.764256477355957 | 1.1157832156733452 | 1.12x slower | 50 | | 21 | 21.18029236793518 | 19.883575677871704 | 0.9387772053597062 | +1.07x faster | 51 | | 22 | 3.8765759468078613 | 1.483001947402954 | 0.382554596569718 | +2.61x faster | 52 | | total | 281.4905562400818 | 183.1653048992157 | 0.650697868325625 | +1.54x faster | 53 | +------------+--------------------+--------------------+---------------------+---------------+ 54 | 55 | -------------------------------------------------------------------------------- /docs/benchmarks/df-ray-0.1.0rc1/m7gd.12xlarge: -------------------------------------------------------------------------------- 1 | ================================================================================= 2 | Spark and DataFusionRay TPCH 100 Benchmarks 3 | Machine: m7gd.12xlarge 4 | Machine On Demand Cost: 2.5628 $/hr 5 | CPU(s): Neoverse-V1 48x 6 | MEM: 189G 7 | HD Throughput: 2153.86 MB/s (from hdparm) 8 | Data Location: /data/sf100 9 | 10 | df-ray duration: 213.06s +1.29x faster 11 | df-ray cost: $0.1517 +1.29x cheaper 12 | 13 | spark duration: 275.49s 14 | spark cost: $0.1961 15 | 16 | DataFusionRay Settings: 17 | concurrency: 16 18 | batch_size : 8192 19 | partitions_per_processor: 2 20 | Ray Workers: 4 21 | Ray Worker Mem (GB): 40 22 | Ray Worker CPU: 10 23 | Ray Head Mem (GB): 10 24 | Ray Head CPU: 4 25 | 26 | Spark Settings: 27 | Executors: 4 28 | Executor Mem (GB): 32 29 | Executor Overhead Mem (GB): 8 30 | Executor CPU: 10 31 | Driver Mem(GB): 10 32 | Driver CPU: 4 33 | ================================================================================= 34 | DataFrame() 35 | +------------+------------+------------+-----------------------+----------------+ 36 | | tpch_query | spark | df_ray | change(=df_ray/spark) | change_text | 37 | +------------+------------+------------+-----------------------+----------------+ 38 | | 1 | 39.8715 | 7.2273 | 0.1813 | +5.52x faster | 39 | | 2 | 5.1250 | 3.3663 | 0.6568 | +1.52x faster | 40 | | 3 | 9.4450 | 7.9609 | 0.8429 | +1.19x faster | 41 | | 4 | 10.7174 | 2.9885 | 0.2789 | +3.59x faster | 42 | | 5 | 17.1354 | 16.6721 | 0.9730 | +1.03x faster | 43 | | 6 | 1.6761 | 5.0164 | 2.9928 | 2.99x slower | 44 | | 7 | 6.1539 | 16.4512 | 2.6733 | 2.67x slower | 45 | | 8 | 22.3847 | 12.0402 | 0.5379 | +1.86x faster | 46 | | 9 | 37.1457 | 16.9252 | 0.4556 | +2.19x faster | 47 | | 10 | 7.6453 | 9.7099 | 1.2701 | 1.27x slower | 48 | | 11 | 1.7255 | 2.7529 | 1.5954 | 1.60x slower | 49 | | 12 | 6.2536 | 4.5053 | 0.7204 | +1.39x faster | 50 | | 13 | 8.1595 | 4.9620 | 0.6081 | +1.64x faster | 51 | | 14 | 2.4955 | 3.3134 | 1.3277 | 1.33x slower | 52 | | 15 | 6.9314 | 6.7785 | 0.9779 | +1.02x faster | 53 | | 16 | 3.8568 | 2.3385 | 0.6063 | +1.65x faster | 54 | | 17 | 32.9444 | 17.6122 | 0.5346 | +1.87x faster | 55 | | 18 | 23.3267 | 33.7229 | 1.4457 | 1.45x slower | 56 | | 19 | 3.5956 | 7.6568 | 2.1295 | 2.13x slower | 57 | | 20 | 4.9011 | 5.8706 | 1.1978 | 1.20x slower | 58 | | 21 | 20.1347 | 23.3859 | 1.1615 | 1.16x slower | 59 | | 22 | 3.8665 | 1.8024 | 0.4662 | +2.15x faster | 60 | | total | 275.4912 | 213.0596 | 0.7734 | +1.29x faster | 61 | +------------+------------+------------+-----------------------+----------------+ 62 | 63 | 64 | ========================================================================================== 65 | Spark and DataFusion for Ray TPCH 100 Benchmarks 66 | Machine: m7gd.12xlarge 67 | CPU(s): 48x 68 | MEM: 189G 69 | HD Throughput: 2162.93 (from hdparm) 70 | 71 | DataFusion for Ray Settings: 72 | concurrency: 16 73 | batch_size : 8192 74 | partitions_per_processor: 2 75 | Ray Workers: 4 76 | Ray Worker Mem (GB): 40 77 | Ray Worker CPU: 10 78 | Ray Head Mem (GB): 10 79 | Ray Head CPU: 4 80 | 81 | Spark Settings: 82 | Executors: 4 83 | Executor Mem (GB): 32 84 | Executor Overhead Mem (GB): 8 85 | Executor CPU: 10 86 | Driver Mem(GB): 10 87 | Driver CPU: 4 88 | ========================================================================================== 89 | DataFrame() 90 | +------------+--------------------+--------------------+---------------------+---------------+ 91 | | tpch_query | spark | df_ray | change | change_text | 92 | +------------+--------------------+--------------------+---------------------+---------------+ 93 | | 1 | 40.10749578475952 | 7.07772159576416 | 0.17646879859434228 | +5.67x faster | 94 | | 2 | 5.265030860900879 | 4.421863079071045 | 0.8398551111843696 | +1.19x faster | 95 | | 3 | 8.993131875991821 | 4.821190357208252 | 0.5360969263754447 | +1.87x faster | 96 | | 4 | 8.75968074798584 | 2.8997082710266113 | 0.3310289900340668 | +3.02x faster | 97 | | 5 | 19.33255624771118 | 10.249737024307251 | 0.5301801217063955 | +1.89x faster | 98 | | 6 | 1.754713773727417 | 4.5369789600372314 | 2.5855948861674696 | 2.59x slower | 99 | | 7 | 6.20657753944397 | 13.625978946685791 | 2.1954094442694267 | 2.20x slower | 100 | | 8 | 24.33410143852234 | 9.368516683578491 | 0.3849953821901789 | +2.60x faster | 101 | | 9 | 38.12766456604004 | 13.534857511520386 | 0.35498784584817605 | +2.82x faster | 102 | | 10 | 7.784897565841675 | 8.692375421524048 | 1.1165690168698141 | 1.12x slower | 103 | | 11 | 2.0291600227355957 | 3.2606608867645264 | 1.6069017969162889 | 1.61x slower | 104 | | 12 | 6.192300081253052 | 4.058101415634155 | 0.6553463757223104 | +1.53x faster | 105 | | 13 | 8.62465763092041 | 3.507314682006836 | 0.40666132292981705 | +2.46x faster | 106 | | 14 | 2.430983304977417 | 3.096514940261841 | 1.2737705495228016 | 1.27x slower | 107 | | 15 | 6.798931360244751 | 5.77943229675293 | 0.8500501020714643 | +1.18x faster | 108 | | 16 | 4.050042390823364 | 3.0604259967803955 | 0.7556528306258588 | +1.32x faster | 109 | | 17 | 33.42715358734131 | 19.431111812591553 | 0.5812972307624187 | +1.72x faster | 110 | | 18 | 23.402374744415283 | 28.70749068260193 | 1.2266913506054615 | 1.23x slower | 111 | | 19 | 3.6461265087127686 | 5.904490232467651 | 1.6193870998053157 | 1.62x slower | 112 | | 20 | 5.166107892990112 | 5.764256477355957 | 1.1157832156733452 | 1.12x slower | 113 | | 21 | 21.18029236793518 | 19.883575677871704 | 0.9387772053597062 | +1.07x faster | 114 | | 22 | 3.8765759468078613 | 1.483001947402954 | 0.382554596569718 | +2.61x faster | 115 | | total | 281.4905562400818 | 183.1653048992157 | 0.650697868325625 | +1.54x faster | 116 | +------------+--------------------+--------------------+---------------------+---------------+ 117 | 118 | -------------------------------------------------------------------------------- /docs/benchmarks/df-ray-0.1.0rc1/r8g.8xlarge: -------------------------------------------------------------------------------- 1 | ========================================================================================== 2 | Spark and DataFusion for Ray TPCH 100 Benchmarks 3 | Machine: r8g.8xlarge 4 | CPU(s): 32x 5 | MEM: 252G 6 | HD Throughput: 167.50 (from hdparm) 7 | 8 | DataFusion for Ray Settings: 9 | concurrency: 16 10 | batch_size : 8192 11 | partitions_per_processor: 8 12 | Ray Workers: 4 13 | Ray Worker Mem (GB): 56 14 | Ray Worker CPU: 6 15 | Ray Head Mem (GB): 8 16 | Ray Head CPU: 4 17 | 18 | Spark Settings: 19 | Executors: 4 20 | Executor Mem (GB): 44 21 | Executor Overhead Mem (GB): 12 22 | Executor CPU: 6 23 | Driver Mem(GB): 8 24 | Driver CPU: 4 25 | ========================================================================================== 26 | DataFrame() 27 | +------------+--------------------+--------------------+---------------------+---------------+ 28 | | tpch_query | spark | df_ray | change | change_text | 29 | +------------+--------------------+--------------------+---------------------+---------------+ 30 | | 1 | 43.84827923774719 | 10.000967502593994 | 0.22808118531558177 | +4.38x faster | 31 | | 2 | 5.0717902183532715 | 3.8610997200012207 | 0.7612893187161155 | +1.31x faster | 32 | | 3 | 10.256386041641235 | 8.126128196716309 | 0.7922993697510979 | +1.26x faster | 33 | | 4 | 8.208275079727173 | 4.202239036560059 | 0.5119515361928797 | +1.95x faster | 34 | | 5 | 36.95858907699585 | 19.0570650100708 | 0.5156329147297589 | +1.94x faster | 35 | | 6 | 1.982734203338623 | 6.400916337966919 | 3.2283279963541 | 3.23x slower | 36 | | 7 | 9.155696153640747 | 18.585452795028687 | 2.029933331463628 | 2.03x slower | 37 | | 8 | 82.31400609016418 | 16.519661903381348 | 0.20069077776734895 | +4.98x faster | 38 | | 9 | 108.94961452484131 | 21.316840410232544 | 0.19565778642908505 | +5.11x faster | 39 | | 10 | 35.651164054870605 | 16.961328268051147 | 0.47575804935698635 | +2.10x faster | 40 | | 11 | 2.080925941467285 | 4.405801773071289 | 2.1172314138026014 | 2.12x slower | 41 | | 12 | 8.010701417922974 | 5.209590673446655 | 0.6503289040071856 | +1.54x faster | 42 | | 13 | 17.210007429122925 | 5.6545422077178955 | 0.3285612880183439 | +3.04x faster | 43 | | 14 | 2.8084030151367188 | 4.262192010879517 | 1.517656827708549 | 1.52x slower | 44 | | 15 | 8.712156772613525 | 7.276721239089966 | 0.8352376373625622 | +1.20x faster | 45 | | 16 | 4.409867525100708 | 4.079038619995117 | 0.9249798541061535 | +1.08x faster | 46 | | 17 | 70.58250260353088 | 26.28227400779724 | 0.37236245582602046 | +2.69x faster | 47 | | 18 | 44.79290056228638 | 57.51821994781494 | 1.2840923277079028 | 1.28x slower | 48 | | 19 | 5.416426420211792 | 7.488842725753784 | 1.382616903611691 | 1.38x slower | 49 | | 20 | 5.868547201156616 | 7.078382253646851 | 1.2061558016014238 | 1.21x slower | 50 | | 21 | 40.45992636680603 | 36.64148950576782 | 0.9056242261436512 | +1.10x faster | 51 | | 22 | 5.988753080368042 | 2.0949254035949707 | 0.34980994799442056 | +2.86x faster | 52 | | total | 558.7376530170441 | 293.0237195491791 | 0.5244388273582853 | +1.91x faster | 53 | +------------+--------------------+--------------------+---------------------+---------------+ 54 | 55 | ========================================================================================== 56 | Spark and DataFusion for Ray TPCH 100 Benchmarks 57 | Machine: r8g.8xlarge 58 | CPU(s): 32x 59 | MEM: 252G 60 | HD Throughput: 169.07 (from hdparm) 61 | 62 | DataFusion for Ray Settings: 63 | concurrency: 16 64 | batch_size : 8192 65 | partitions_per_processor: 8 66 | Ray Workers: 4 67 | Ray Worker Mem (GB): 56 68 | Ray Worker CPU: 6 69 | Ray Head Mem (GB): 8 70 | Ray Head CPU: 4 71 | 72 | Spark Settings: 73 | Executors: 4 74 | Executor Mem (GB): 44 75 | Executor Overhead Mem (GB): 12 76 | Executor CPU: 6 77 | Driver Mem(GB): 8 78 | Driver CPU: 4 79 | ========================================================================================== 80 | DataFrame() 81 | +------------+--------------------+--------------------+---------------------+---------------+ 82 | | tpch_query | spark | df_ray | change | change_text | 83 | +------------+--------------------+--------------------+---------------------+---------------+ 84 | | 1 | 44.023725271224976 | 10.511452674865723 | 0.2387679054897309 | +4.19x faster | 85 | | 2 | 5.121235370635986 | 4.563743352890015 | 0.8911411061201159 | +1.12x faster | 86 | | 3 | 10.236863613128662 | 11.815351963043213 | 1.1541964814193828 | 1.15x slower | 87 | | 4 | 8.071234464645386 | 6.747247695922852 | 0.8359622961615074 | +1.20x faster | 88 | | 5 | 36.89992165565491 | 15.730475664138794 | 0.42630105860206075 | +2.35x faster | 89 | | 6 | 1.977816104888916 | 6.389086723327637 | 3.230374506272149 | 3.23x slower | 90 | | 7 | 9.170192956924438 | 19.880183219909668 | 2.1679132940052352 | 2.17x slower | 91 | | 8 | 91.3105399608612 | 12.09617304801941 | 0.1324729111579478 | +7.55x faster | 92 | | 9 | 106.27129793167114 | 21.19954752922058 | 0.19948516619088602 | +5.01x faster | 93 | | 10 | 25.633528232574463 | 14.167945146560669 | 0.5527114729589346 | +1.81x faster | 94 | | 11 | 1.7444405555725098 | 4.269078493118286 | 2.4472479039087767 | 2.45x slower | 95 | | 12 | 7.713707447052002 | 9.96561598777771 | 1.2919359537787933 | 1.29x slower | 96 | | 13 | 17.422370433807373 | 5.676977157592773 | 0.32584413120827915 | +3.07x faster | 97 | | 14 | 2.8041045665740967 | 3.8914337158203125 | 1.3877634101836138 | 1.39x slower | 98 | | 15 | 8.518751859664917 | 10.617808103561401 | 1.2464042008120013 | 1.25x slower | 99 | | 16 | 4.334302186965942 | 4.8702826499938965 | 1.1236601510249442 | 1.12x slower | 100 | | 17 | 67.09746074676514 | 31.08197546005249 | 0.46323624044969536 | +2.16x faster | 101 | | 18 | 70.05180883407593 | 56.567989110946655 | 0.807516466062041 | +1.24x faster | 102 | | 19 | 5.464143991470337 | 7.1256444454193115 | 1.304073329059889 | 1.30x slower | 103 | | 20 | 5.807567119598389 | 7.7169201374053955 | 1.3287698580983502 | 1.33x slower | 104 | | 21 | 39.69502830505371 | 28.762243032455444 | 0.7245804893101342 | +1.38x faster | 105 | | 22 | 5.841529130935669 | 2.004164695739746 | 0.34308905268075385 | +2.91x faster | 106 | | total | 575.2115707397461 | 295.651340007782 | 0.5139871223862239 | +1.95x faster | 107 | +------------+--------------------+--------------------+---------------------+---------------+ 108 | -------------------------------------------------------------------------------- /docs/contributing.md: -------------------------------------------------------------------------------- 1 | 19 | 20 | # DataFusion for Ray Contributor Guide 21 | 22 | ## Building 23 | 24 | You'll need to have both rust and cargo installed. 25 | 26 | We will follow the development workflow outlined by [datafusion-python](https://github.com/apache/datafusion-python), [pyo3](https://github.com/PyO3/pyo3) and [maturin](https://github.com/PyO3/maturin). 27 | 28 | The Maturin tools used in this workflow can be installed either via `uv` or `pip`. Both approaches should offer the same experience. It is recommended to use `uv` since it has significant performance improvements 29 | over `pip`. 30 | 31 | Bootstrap (`uv`): 32 | 33 | By default `uv` will attempt to build the datafusion-ray python package. For our development we prefer to build manually. This means 34 | that when creating your virtual environment using `uv sync` you need to pass in the additional `--no-install-package datafusion-ray`. This tells uv, to install all of the dependencies found in `pyproject.toml`, but skip building `datafusion-ray` as we'll do that manually. 35 | 36 | ```bash 37 | # fetch this repo 38 | git clone git@github.com:apache/datafusion-ray.git 39 | # go to repo root 40 | cd datafusion-ray 41 | # create the virtual enviornment 42 | uv sync --dev --no-install-package datafusion-ray 43 | # activate the environment 44 | source .venv/bin/activate 45 | ``` 46 | 47 | Bootstrap (`pip`): 48 | 49 | ```bash 50 | # fetch this repo 51 | git clone git@github.com:apache/datafusion-python.git 52 | # go to repo root 53 | cd datafusion-ray 54 | # prepare development environment (used to build wheel / install in development) 55 | python3 -m venv .venv 56 | # activate the venv 57 | source .venv/bin/activate 58 | # update pip itself if necessary 59 | python -m pip install -U pip 60 | # install dependencies 61 | python -m pip install -r pyproject.toml 62 | ``` 63 | 64 | Whenever rust code changes (your changes or via `git pull`): 65 | 66 | ```bash 67 | # make sure you activate the venv using "source venv/bin/activate" first 68 | maturin develop --uv 69 | python -m pytest 70 | ``` 71 | 72 | ## Example 73 | 74 | - In the `examples` directory, run 75 | 76 | ```bash 77 | RAY_COLOR_PREFIX=1 RAY_DEDUP_LOGS=0 python tips.py --data-dir=$(pwd)/../testdata/tips/ 78 | ``` 79 | 80 | - In the `tpch` directory, use `make_data.py` to create a TPCH dataset at a provided scale factor, then 81 | 82 | ```bash 83 | RAY_COLOR_PREFIX=1 RAY_DEDUP_LOGS=0 python tpc.py --data=file:///path/to/your/tpch/directory/ --concurrency=2 --batch-size=8182 --worker-pool-min=10 --qnum 2 84 | ``` 85 | 86 | To execute the TPCH query #2. To execute an arbitrary query against the TPCH dataset, provide it with `--query` instead of `--qnum`. This is useful for validating plans that DataFusion for Ray will create. 87 | 88 | For example, to execute the following query: 89 | 90 | ```bash 91 | RAY_COLOR_PREFIX=1 RAY_DEDUP_LOGS=0 python tpc.py --data=file:///path/to/your/tpch/directory/ --concurrency=2 --batch-size=8182 --worker-pool-min=10 --query 'select c.c_name, sum(o.o_totalprice) as total from orders o inner join customer c on o.o_custkey = c.c_custkey group by c_name limit 1' 92 | ``` 93 | 94 | To further parallelize execution, you can choose how many partitions will be served by each Stage with `--partitions-per-processor`. If this number is less than `--concurrency` Then multiple Actors will host portions of the stage. For example, if there are 10 stages calculated for a query, `concurrency=16` and `partitions-per-processor=4`, then `40` `RayStage` Actors will be created. If `partitions-per-processor=16` or is absent, then `10` `RayStage` Actors will be created. 95 | 96 | To validate the output against non-ray single node datafusion, add `--validate` which will ensure that both systems produce the same output. 97 | 98 | To run the entire TPCH benchmark use 99 | 100 | ```bash 101 | RAY_COLOR_PREFIX=1 RAY_DEDUP_LOGS=0 python tpcbench.py --data=file:///path/to/your/tpch/directory/ --concurrency=2 --batch-size=8182 --worker-pool-min=10 [--partitions-per-processor=] [--validate] 102 | ``` 103 | 104 | This will output a json file in the current directory with query timings. 105 | 106 | ## Logging 107 | 108 | DataFusion for Ray's logging output is determined by the `DATAFUSION_RAY_LOG_LEVEL` environment variable. The default log level is `WARN`. To change the log level, set the environment variable to one of the following values: `ERROR`, `WARN`, `INFO`, `DEBUG`, or `TRACE`. 109 | 110 | DataFusion for Ray outputs logs from both python and rust, and in order to handle this consistently, the python logger for `datafusion_ray` is routed to rust for logging. The `RUST_LOG` environment variable can be used to control other rust log output other than `datafusion_ray`. 111 | 112 | ## Status 113 | 114 | - DataFusion for Ray can execute all TPCH queries. Tested up to SF100. 115 | 116 | ## Known Issues 117 | 118 | - We are waiting to upgrade to a DataFusion version where the parquet options are serialized into substrait in order to send them correctly in a plan. Currently, we 119 | manually add back `table_parquet_options.pushdown_filters=true` after deserialization to compensate. This will be refactored in the future. 120 | 121 | see 122 | -------------------------------------------------------------------------------- /examples/http_csv.py: -------------------------------------------------------------------------------- 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 | # this is a port of the example at 19 | # https://github.com/apache/datafusion/blob/45.0.0/datafusion-examples/examples/query-http-csv.rs 20 | 21 | import ray 22 | 23 | from datafusion_ray import DFRayContext, df_ray_runtime_env 24 | 25 | 26 | def main(): 27 | ctx = DFRayContext() 28 | ctx.register_csv( 29 | "aggregate_test_100", 30 | "https://github.com/apache/arrow-testing/raw/master/data/csv/aggregate_test_100.csv", 31 | ) 32 | 33 | df = ctx.sql("SELECT c1,c2,c3 FROM aggregate_test_100 LIMIT 5") 34 | 35 | df.show() 36 | 37 | 38 | if __name__ == __name__: 39 | ray.init(namespace="http_csv", runtime_env=df_ray_runtime_env) 40 | main() 41 | -------------------------------------------------------------------------------- /examples/tips.py: -------------------------------------------------------------------------------- 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 | import argparse 19 | import os 20 | import ray 21 | 22 | from datafusion_ray import DFRayContext, df_ray_runtime_env 23 | 24 | 25 | def go(data_dir: str): 26 | ctx = DFRayContext() 27 | 28 | ctx.register_parquet("tips", os.path.join(data_dir, "tips.parquet")) 29 | 30 | df = ctx.sql( 31 | "select sex, smoker, avg(tip/total_bill) as tip_pct from tips group by sex, smoker order by sex, smoker" 32 | ) 33 | df.show() 34 | 35 | 36 | if __name__ == "__main__": 37 | ray.init(namespace="tips", runtime_env=df_ray_runtime_env) 38 | parser = argparse.ArgumentParser() 39 | parser.add_argument( 40 | "--data-dir", required=True, help="path to tips.parquet files" 41 | ) 42 | args = parser.parse_args() 43 | 44 | go(args.data_dir) 45 | -------------------------------------------------------------------------------- /k8s/README.md: -------------------------------------------------------------------------------- 1 | ## Benchmarking on kubernetes 2 | 3 | This directory contains a utility `bench_toolbox.py` to facilitate benchmarking spark and datafusion-ray on k8s clusters. 4 | 5 | The paved path is to execute the steps on a fresh 24.04 ubuntu ami, but the tool should also work on for established k8s setups. 6 | 7 | If that is the case you'll want to skip the install of `https://k3s.io/` from the `k3s` subcommand and proceed. The `machine_prep.sh` script should provide clues about the environment requirements you'll need to satisfy to operate th tool. 8 | 9 | ### Current status 10 | 11 | Help wanted! This code is rough in that it has too many steps and doesn't handle enough variety in machine/architecture to run unattended. PRs welcome to improve any and all of what is here. 12 | 13 | #### Known issues and quirks 14 | 15 | - These instructions are likely incomplete 16 | - benchmark results for df_ray and spark do not ensure the same settings are used, they require the operator use the same versions 17 | - the results subcommand just looks for the newest DataFusion for Ray and Spark results, and assumes those are the correct ones 18 | - the machine_prep script does not handle errors 19 | 20 | ### Sample results 21 | 22 | You can find results conducted with different versions of DataFusion for Ray in `docs/benchmarks/[dfray version]` 23 | 24 | ### Benchmarking on a fresh ubuntu 24.04 LTS ami from amazon 25 | 26 | - provision the machine and ssh in and download the repo 27 | 28 | ```bash 29 | git checkout https://github.com/robtandy/datafusion-ray 30 | cd datafusion-ray 31 | git checkout k8s_benchmarking 32 | ``` 33 | 34 | then run the machine prep script 35 | 36 | ````bash 37 | cd datafusion-ray/k8s 38 | ```bash 39 | ./machine_prep.sh 40 | ```` 41 | 42 | Next, you'll want to choose where you'll keep your TPCH data. 43 | 44 | ```bash 45 | sudo mkdir /data 46 | sudo chmod -R 777 /data 47 | ``` 48 | 49 | At this point, you'll have the configuration needed to operate the `bench_toolbox.py` script. So, if you first need kubernetes installed, run 50 | 51 | ```bash 52 | ./bench_toolbox.py -v k3s --data-path /data 53 | ``` 54 | 55 | This will: 56 | 57 | - create a single machine cluster using k3s 58 | - create the PVC for /data 59 | - install kuberay operater 60 | - install spark operator 61 | 62 | Next, lets generate TPCH data in the /data directory, scale factor 1 to start 63 | 64 | ```bash 65 | ./bench_toolbox.py -v generate --data-path /data --scale-factor 1 --partitions 2 --pool-size 2 66 | ``` 67 | 68 | Now we can run a benchmark with the generated data with DataFusion for Ray 69 | 70 | ```bash 71 | ./bench_toolbox.py -v bench --executor-cpus 2 --executor-mem 10 --executor-num 2 --executor-overhead-mem 4 --driver-mem 4 --driver-cpus 2 --data-path /data --concurrency 8 --partitions-per-processor 4 --processor-pool-min 50 --df-ray-version 0.1.0rc1 --test-pypi --arm --scale-factor 1 --output-path /data df_ray 72 | ``` 73 | 74 | followed by spark. Make sure you use the same settings 75 | 76 | ```bash 77 | ./bench_toolbox.py -v bench --executor-cpus 2 --executor-mem 10 --executor-num 2 --executor-overhead-mem 4 --driver-mem 4 --driver-cpus 2 --data-path /data --concurrency 8 --partitions-per-processor 4 --processor-pool-min 50 --df-ray-version 0.1.0rc1 --test-pypi --arm --scale-factor 1 --output-path /data spark 78 | ``` 79 | 80 | Lastly, compile the results: 81 | 82 | ```bash 83 | ./bench_toolbox.py -v results --data-device /dev/nvme1n1 --data-path /data --scale-factor 1 --output-path /data 84 | ``` 85 | 86 | You should get a table of results similar to what you'd find in `docs/benchmarks` 87 | -------------------------------------------------------------------------------- /k8s/machine_prep.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | sudo apt-get update 4 | 5 | # install docker 6 | sudo apt-get install -y ca-certificates curl 7 | sudo install -m 0755 -d /etc/apt/keyrings 8 | sudo curl -fsSL https://download.docker.com/linux/ubuntu/gpg -o /etc/apt/keyrings/docker.asc 9 | sudo chmod a+r /etc/apt/keyrings/docker.asc 10 | 11 | # Add the repository to Apt sources: 12 | echo \ 13 | "deb [arch=$(dpkg --print-architecture) signed-by=/etc/apt/keyrings/docker.asc] https://download.docker.com/linux/ubuntu \ 14 | $(. /etc/os-release && echo "${UBUNTU_CODENAME:-$VERSION_CODENAME}") stable" | 15 | sudo tee /etc/apt/sources.list.d/docker.list >/dev/null 16 | sudo apt-get update 17 | 18 | sudo apt-get install -y docker-ce docker-ce-cli containerd.io docker-buildx-plugin docker-compose-plugin 19 | sudo groupadd docker 20 | sudo usermod -aG docker $USER 21 | newgrp docker 22 | # end docker section 23 | 24 | sudo apt install -y python3-virtualenv 25 | 26 | virtualenv -p $(which python3) venv 27 | . ./venv/bin/activate 28 | # install what we need to run the tool 29 | pip install click jinja2 pandas datafusion 'ray[default]' ec2_metadata duckdb boto3 30 | -------------------------------------------------------------------------------- /k8s/pricing.py: -------------------------------------------------------------------------------- 1 | import boto3 2 | import json 3 | 4 | region_mapping = { 5 | "ap-east-1": "Asia Pacific (Hong Kong)", 6 | "ap-southeast-1": "Asia Pacific (Singapore)", 7 | "ap-southeast-2": "Asia Pacific (Sydney)", 8 | "ap-northeast-1": "Asia Pacific (Tokyo)", 9 | "ap-northeast-2": "Asia Pacific (Seoul)", 10 | "ap-northeast-3": "Asia Pacific (Osaka-Local)", 11 | "ap-south-1": "Asia Pacific (Mumbai)", 12 | "ca-central-1": "Canada (Central)", 13 | "eu-central-1": "EU (Frankfurt)", 14 | "eu-west-1": "EU (Ireland)", 15 | "eu-west-2": "EU (London)", 16 | "eu-west-3": "EU (Paris)", 17 | "sa-east-1": "South America (Sao Paulo)", 18 | "us-east-1": "US East (N. Virginia)", 19 | "us-west-1": "US West (N. California)", 20 | "us-east-2": "US East (Ohio)", 21 | "us-west-2": "US West (Oregon)", 22 | "cn-north-1": "China (Beijing)", 23 | "cn-northwest-1": "China (Ningxia)", 24 | "us-gov-west-1": "AWS GovCloud (US)", 25 | } 26 | 27 | 28 | def get_on_demand_price(region, instance_type): 29 | """Get on-demand price for a specific instance type in a region.""" 30 | pricing_client = boto3.client("pricing", region_name="us-east-1") 31 | 32 | try: 33 | response = pricing_client.get_products( 34 | ServiceCode="AmazonEC2", 35 | Filters=[ 36 | { 37 | "Type": "TERM_MATCH", 38 | "Field": "instanceType", 39 | "Value": instance_type, 40 | }, 41 | { 42 | "Type": "TERM_MATCH", 43 | "Field": "operatingSystem", 44 | "Value": "Linux", 45 | }, 46 | { 47 | "Type": "TERM_MATCH", 48 | "Field": "preInstalledSw", 49 | "Value": "NA", 50 | }, 51 | {"Type": "TERM_MATCH", "Field": "tenancy", "Value": "Shared"}, 52 | { 53 | "Type": "TERM_MATCH", 54 | "Field": "capacitystatus", 55 | "Value": "Used", 56 | }, 57 | { 58 | "Type": "TERM_MATCH", 59 | "Field": "location", 60 | "Value": region_mapping[region], 61 | }, 62 | ], 63 | ) 64 | 65 | if response["PriceList"]: 66 | price_data = eval(response["PriceList"][0]) 67 | terms = price_data["terms"]["OnDemand"] 68 | price_dimensions = next(iter(terms.values()))["priceDimensions"] 69 | price = next(iter(price_dimensions.values()))["pricePerUnit"][ 70 | "USD" 71 | ] 72 | return float(price) 73 | except Exception as e: 74 | print( 75 | f"Error getting on-demand price for {instance_type} in {region}: {str(e)}" 76 | ) 77 | return None 78 | 79 | 80 | def get_reserved(region, instance_type): 81 | """Get savings plans rates for a specific instance type in a region.""" 82 | savingsplans_client = boto3.client("savingsplans", region_name="us-east-1") 83 | 84 | def request(): 85 | results = [] 86 | nt = None 87 | while True: 88 | kwargs = dict( 89 | filters=[ 90 | {"name": "instanceType", "values": [instance_type]}, 91 | {"name": "region", "values": [region]}, 92 | {"name": "tenancy", "values": ["shared"]}, 93 | {"name": "productDescription", "values": ["Linux/UNIX"]}, 94 | ], 95 | serviceCodes=["AmazonEC2"], 96 | ) 97 | if nt: 98 | kwargs["nextToken"] = nt 99 | 100 | response = ( 101 | savingsplans_client.describe_savings_plans_offering_rates( 102 | **kwargs 103 | ) 104 | ) 105 | results.extend(response["searchResults"]) 106 | if len(response["nextToken"]) > 0: 107 | nt = response["nextToken"] 108 | else: 109 | break 110 | return results 111 | 112 | try: 113 | results = request() 114 | rates = { 115 | "1.0y": {"no": None, "partial": None, "all": None}, 116 | "3.0y": {"no": None, "partial": None, "all": None}, 117 | } 118 | 119 | for result in results: 120 | # Skip unused box entries 121 | if "UnusedBox" in result["usageType"]: 122 | continue 123 | 124 | duration_seconds = result["savingsPlanOffering"]["durationSeconds"] 125 | duration_years = duration_seconds / (365 * 24 * 60 * 60) 126 | key = f"{duration_years:.1f}y" 127 | 128 | payment_option = ( 129 | result["savingsPlanOffering"]["paymentOption"] 130 | .lower() 131 | .split()[0] 132 | ) # 'no', 'partial', or 'all' 133 | rate = float(result["rate"]) 134 | 135 | if key in rates: 136 | rates[key][payment_option] = rate 137 | 138 | return ( 139 | rates 140 | if any( 141 | any(v is not None for v in year.values()) 142 | for year in rates.values() 143 | ) 144 | else None 145 | ) 146 | except Exception as e: 147 | print( 148 | f"Error getting reserved cost for {instance_type} in {region}: {str(e)}" 149 | ) 150 | return None 151 | -------------------------------------------------------------------------------- /k8s/pvcs.yaml.template: -------------------------------------------------------------------------------- 1 | apiVersion: v1 2 | kind: PersistentVolume 3 | metadata: 4 | name: ray-pv 5 | spec: 6 | storageClassName: local-path 7 | capacity: 8 | storage: 300Gi 9 | volumeMode: Filesystem 10 | hostPath: 11 | path: "{{ data_path }}" 12 | accessModes: 13 | - ReadWriteOnce 14 | claimRef: 15 | name: ray-pvc 16 | namespace: default 17 | --- 18 | apiVersion: v1 19 | kind: PersistentVolumeClaim 20 | metadata: 21 | name: ray-pvc 22 | namespace: default 23 | spec: 24 | storageClassName: local-path 25 | accessModes: 26 | - ReadWriteOnce 27 | volumeName: ray-pv 28 | resources: 29 | requests: 30 | storage: 300Gi 31 | 32 | -------------------------------------------------------------------------------- /k8s/ray_cluster.yaml.template: -------------------------------------------------------------------------------- 1 | apiVersion: ray.io/v1alpha1 2 | kind: RayCluster 3 | metadata: 4 | name: datafusion-ray-cluster 5 | spec: 6 | headGroupSpec: 7 | rayStartParams: 8 | num-cpus: "0" 9 | template: 10 | spec: 11 | containers: 12 | - name: ray-head 13 | image: rayproject/ray:2.43.0-py312-cpu{{ "-aarch64" if arm else "" }} 14 | imagePullPolicy: IfNotPresent 15 | resources: 16 | limits: 17 | cpu: {{ driver_cpus }} 18 | memory: {{ driver_mem }}Gi 19 | requests: 20 | cpu: {{ driver_cpus }} 21 | memory: {{ driver_mem }}Gi 22 | volumeMounts: 23 | - mountPath: /data 24 | name: ray-storage 25 | volumes: 26 | - name: ray-storage 27 | hostPath: 28 | path: /data 29 | type: DirectoryOrCreate 30 | nodeSelector: 31 | node-role.kubernetes.io/master: "true" 32 | workerGroupSpecs: 33 | - replicas: {{ executor_num }} 34 | groupName: "datafusion-ray" 35 | rayStartParams: 36 | num-cpus: "{{ executor_cpus }}" 37 | template: 38 | spec: 39 | containers: 40 | - name: ray-worker 41 | image: rayproject/ray:2.43.0-py312-cpu{{ "-aarch64" if arm else "" }} 42 | imagePullPolicy: IfNotPresent 43 | resources: 44 | limits: 45 | cpu: {{ executor_cpus }} 46 | memory: {{ executor_mem }}Gi 47 | requests: 48 | cpu: {{ executor_cpus }} 49 | memory: {{ executor_mem }}Gi 50 | volumeMounts: 51 | - mountPath: /data/ 52 | name: ray-storage 53 | volumes: 54 | - name: ray-storage 55 | hostPath: 56 | path: /data 57 | type: DirectoryOrCreate 58 | -------------------------------------------------------------------------------- /k8s/ray_job.sh.template: -------------------------------------------------------------------------------- 1 | ray job submit --address='http://localhost:8265' \ 2 | --runtime-env-json='{"pip":"requirements.txt", "working_dir":"./", "env_vars":{"DATAFUSION_RAY_LOG_LEVEL":"WARN", "RAY_OVERRIDE_JOB_RUNTIME_ENV":"1", "RAY_DEDUP_LOGS":"O", "RAY_COLOR_PREFIX":"1"}}' -- \ 3 | python tpcbench.py \ 4 | --data {{ data_path }}/sf{{ scale_factor }} \ 5 | --concurrency {{ concurrency }} \ 6 | --partitions-per-processor {{ partitions_per_processor }} \ 7 | --processor-pool-min {{ processor_pool_min }} \ 8 | --listing-tables \ 9 | --output-path /data 10 | -------------------------------------------------------------------------------- /k8s/requirements.txt.template: -------------------------------------------------------------------------------- 1 | {% if test_pypi %} 2 | --index-url https://test.pypi.org/simple/ 3 | --extra-index-url https://pypi.org/simple/ 4 | {% endif %} 5 | datafusion-ray=={{ df_ray_version }} 6 | duckdb 7 | -------------------------------------------------------------------------------- /k8s/spark_job.yaml.template: -------------------------------------------------------------------------------- 1 | apiVersion: sparkoperator.k8s.io/v1beta2 2 | kind: SparkApplication 3 | metadata: 4 | name: spark-tpch-bench 5 | namespace: default 6 | spec: 7 | type: Python 8 | pythonVersion: "3" 9 | mode: cluster 10 | image: spark:3.5.3 11 | imagePullPolicy: IfNotPresent 12 | # see https://github.com/kubeflow/spark-operator/issues/1132#issuecomment-2688276488 13 | deps: 14 | {% if data_path.startswith("s3") %} 15 | packages: 16 | - org.apache.hadoop:hadoop-aws:3.3.4 17 | - com.amazonaws:aws-java-sdk-bundle:1.12.782 18 | repositories: 19 | - https://repo1.maven.org/maven2/ 20 | {% endif %} 21 | mainApplicationFile: {{ "local://" if data_path.startswith("/") else "" }}{{ data_path }}/spark_tpcbench.py 22 | arguments: 23 | - --name 24 | - spark 25 | - --benchmark 26 | - tpch 27 | - --data 28 | - {{ data_path }}/sf{{ scale_factor }} 29 | - --queries 30 | - /data/queries 31 | - --output 32 | - {{ output_path }} 33 | sparkVersion: 3.5.3 34 | volumes: 35 | - name: spark-local-dir-spark-vol 36 | hostPath: 37 | path: /data 38 | type: DirectoryOrCreate 39 | driver: 40 | javaOptions: "-Divy.cache.dir=/tmp/ivy2/cache -Divy.home=/tmp/ivy2" 41 | cores: {{ driver_cpus }} 42 | memory: "{{ driver_mem }}g" 43 | serviceAccount: spark-operator-spark 44 | volumeMounts: 45 | - name: spark-local-dir-spark-vol 46 | mountPath: /data 47 | nodeSelector: 48 | node-role.kubernetes.io/master: "true" 49 | executor: 50 | javaOptions: "-Divy.cache.dir=/tmp/ivy2/cache -Divy.home=/tmp/ivy2" 51 | instances: {{ executor_num }} 52 | cores: {{ executor_cpus }} 53 | memory: "{{ executor_mem - executor_overhead_mem }}g" 54 | memoryOverhead: "{{ executor_overhead_mem }}g" 55 | volumeMounts: 56 | - name: spark-local-dir-spark-vol 57 | mountPath: /data 58 | 59 | -------------------------------------------------------------------------------- /k8s/spark_tpcbench.py: -------------------------------------------------------------------------------- 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 | import argparse 19 | from datetime import datetime 20 | import json 21 | from pyspark.sql import SparkSession 22 | import time 23 | import sys 24 | 25 | 26 | def main(benchmark: str, data_path: str, query_path: str, output_path: str, name: str): 27 | 28 | # Initialize a SparkSession 29 | spark = SparkSession.builder \ 30 | .appName( f"{name} benchmark derived from {benchmark}") \ 31 | .getOrCreate() 32 | 33 | spark.conf.set("spark.hadoop.fs.s3a.aws.credentials.provider", "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider") 34 | spark.conf.set("spark.hadoop.fs.s3a.impl", "org.apache.hadoop.fs.s3a.S3AFileSystem") 35 | 36 | # Register the tables 37 | num_queries = 22 38 | table_names = [ 39 | "customer", 40 | "lineitem", 41 | "nation", 42 | "orders", 43 | "part", 44 | "partsupp", 45 | "region", 46 | "supplier", 47 | ] 48 | 49 | for table in table_names: 50 | path = f"{data_path}/{table}.parquet" 51 | print(f"Registering table {table} using path {path}") 52 | df = spark.read.parquet(path) 53 | df.createOrReplaceTempView(table) 54 | 55 | conf_dict = {k: v for k, v in spark.sparkContext.getConf().getAll()} 56 | 57 | results = { 58 | "engine": "spark", 59 | "benchmark": benchmark, 60 | "data_path": data_path, 61 | "query_path": query_path, 62 | "spark_conf": conf_dict, 63 | "queries": {}, 64 | } 65 | 66 | iter_start_time = time.time() 67 | 68 | for query in range(1, num_queries + 1): 69 | spark.sparkContext.setJobDescription(f"{benchmark} q{query}") 70 | 71 | # if query == 9: 72 | # continue 73 | 74 | # read text file 75 | path = f"{query_path}/q{query}.sql" 76 | 77 | # if query == 72: 78 | # # use version with sensible join order 79 | # path = f"{query_path}/q{query}_optimized.sql" 80 | 81 | print(f"Reading query {query} using path {path}") 82 | with open(path, "r") as f: 83 | text = f.read() 84 | # each file can contain multiple queries 85 | queries = list( 86 | filter(lambda x: len(x) > 0, map(lambda x: x.strip(), text.split(";"))) 87 | ) 88 | 89 | start_time = time.time() 90 | for sql in queries: 91 | sql = sql.strip().replace("create view", "create temp view") 92 | if len(sql) > 0: 93 | print(f"Executing: {sql}") 94 | df = spark.sql(sql) 95 | rows = df.collect() 96 | end_time = time.time() 97 | 98 | out_path = f"{output_path}/{name}_{benchmark}_q{query}_result.txt" 99 | # fIXME: concat output for all queries. For example q15 has multiple 100 | out = df._show_string(100000) 101 | with open(out_path, "w") as f: 102 | f.write(out) 103 | 104 | print(f"Query {query} took {end_time - start_time} seconds") 105 | 106 | results["queries"][str(query)] = end_time - start_time 107 | print(json.dumps(results, indent=4)) 108 | 109 | iter_end_time = time.time() 110 | print(f"total took {round(iter_end_time - iter_start_time,2)} seconds") 111 | 112 | out = json.dumps(results, indent=4) 113 | current_time_millis = int(datetime.now().timestamp() * 1000) 114 | results_path = f"{output_path}/{name}-{benchmark}-{current_time_millis}.json" 115 | print(f"Writing results to {results_path}") 116 | with open(results_path, "w") as f: 117 | f.write(out) 118 | 119 | # Stop the SparkSession 120 | spark.stop() 121 | 122 | 123 | if __name__ == "__main__": 124 | print(f"got arguments {sys.argv}") 125 | print(f"python version {sys.version}") 126 | print(f"python versioninfo {sys.version_info}") 127 | 128 | parser = argparse.ArgumentParser( 129 | description="DataFusion benchmark derived from TPC-H / TPC-DS" 130 | ) 131 | parser.add_argument( 132 | "--benchmark", required=True, help="Benchmark to run (tpch or tpcds)" 133 | ) 134 | parser.add_argument("--data", required=True, help="Path to data files") 135 | parser.add_argument("--queries", required=True, help="Path to query files") 136 | parser.add_argument("--output", required=True, help="Path to write output") 137 | parser.add_argument( 138 | "--name", required=True, help="Prefix for result file e.g. spark/comet/gluten" 139 | ) 140 | args = parser.parse_args() 141 | print(f"parsed is {args}") 142 | 143 | main(args.benchmark, args.data, args.queries, args.output, args.name) 144 | -------------------------------------------------------------------------------- /pyproject.toml: -------------------------------------------------------------------------------- 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 | [build-system] 19 | requires = ["maturin>=1.8.1,<2.0"] 20 | build-backend = "maturin" 21 | 22 | [project] 23 | name = "datafusion-ray" 24 | requires-python = ">=3.10,<3.13" 25 | classifiers = [ 26 | "Programming Language :: Rust", 27 | "Programming Language :: Python :: Implementation :: CPython", 28 | "Programming Language :: Python :: Implementation :: PyPy", 29 | ] 30 | keywords = [ 31 | "datafusion", 32 | "ray", 33 | "dataframe", 34 | "rust", 35 | "query-engine", 36 | "distributed", 37 | ] 38 | dependencies = [ 39 | "ray[default]==2.43.0", 40 | "pyarrow>=18.0.0", 41 | "typing-extensions;python_version<'3.13'", 42 | ] 43 | dynamic = ["version"] 44 | 45 | [tool.maturin] 46 | module-name = "datafusion_ray._datafusion_ray_internal" 47 | 48 | [dependency-groups] 49 | dev = ["maturin>=1.8.1", "duckdb", "datafusion>=43.0.0"] 50 | -------------------------------------------------------------------------------- /src/codec.rs: -------------------------------------------------------------------------------- 1 | use std::sync::Arc; 2 | 3 | use crate::{ 4 | isolator::PartitionIsolatorExec, 5 | max_rows::MaxRowsExec, 6 | pre_fetch::PrefetchExec, 7 | protobuf::{ 8 | DfRayStageReaderExecNode, MaxRowsExecNode, PartitionIsolatorExecNode, PrefetchExecNode, 9 | }, 10 | }; 11 | 12 | use arrow::datatypes::Schema; 13 | use datafusion::{ 14 | common::{internal_datafusion_err, internal_err}, 15 | error::Result, 16 | execution::FunctionRegistry, 17 | physical_plan::ExecutionPlan, 18 | }; 19 | use datafusion_proto::physical_plan::{ 20 | DefaultPhysicalExtensionCodec, PhysicalExtensionCodec, from_proto::parse_protobuf_partitioning, 21 | to_proto::serialize_partitioning, 22 | }; 23 | use datafusion_proto::protobuf; 24 | 25 | use prost::Message; 26 | 27 | use crate::stage_reader::DFRayStageReaderExec; 28 | 29 | #[derive(Debug)] 30 | /// Physical Extension Codec for for DataFusion for Ray plans 31 | pub struct RayCodec {} 32 | 33 | impl PhysicalExtensionCodec for RayCodec { 34 | fn try_decode( 35 | &self, 36 | buf: &[u8], 37 | inputs: &[Arc], 38 | registry: &dyn FunctionRegistry, 39 | ) -> Result> { 40 | // TODO: clean this up 41 | if let Ok(node) = PartitionIsolatorExecNode::decode(buf) { 42 | if inputs.len() != 1 { 43 | Err(internal_datafusion_err!( 44 | "PartitionIsolatorExec requires one input" 45 | )) 46 | } else { 47 | Ok(Arc::new(PartitionIsolatorExec::new( 48 | inputs[0].clone(), 49 | node.partition_count as usize, 50 | ))) 51 | } 52 | } else if let Ok(node) = DfRayStageReaderExecNode::decode(buf) { 53 | let schema: Schema = node 54 | .schema 55 | .as_ref() 56 | .ok_or(internal_datafusion_err!("missing schema in proto"))? 57 | .try_into()?; 58 | 59 | let part = parse_protobuf_partitioning( 60 | node.partitioning.as_ref(), 61 | registry, 62 | &schema, 63 | &DefaultPhysicalExtensionCodec {}, 64 | )? 65 | .ok_or(internal_datafusion_err!("missing partitioning in proto"))?; 66 | 67 | Ok(Arc::new(DFRayStageReaderExec::try_new( 68 | part, 69 | Arc::new(schema), 70 | node.stage_id as usize, 71 | )?)) 72 | } else if let Ok(node) = MaxRowsExecNode::decode(buf) { 73 | if inputs.len() != 1 { 74 | Err(internal_datafusion_err!( 75 | "MaxRowsExec requires one input, got {}", 76 | inputs.len() 77 | )) 78 | } else { 79 | Ok(Arc::new(MaxRowsExec::new( 80 | inputs[0].clone(), 81 | node.max_rows as usize, 82 | ))) 83 | } 84 | } else if let Ok(node) = PrefetchExecNode::decode(buf) { 85 | if inputs.len() != 1 { 86 | Err(internal_datafusion_err!( 87 | "MaxRowsExec requires one input, got {}", 88 | inputs.len() 89 | )) 90 | } else { 91 | Ok(Arc::new(PrefetchExec::new( 92 | inputs[0].clone(), 93 | node.buf_size as usize, 94 | ))) 95 | } 96 | } else { 97 | internal_err!("Should not reach this point") 98 | } 99 | } 100 | 101 | fn try_encode(&self, node: Arc, buf: &mut Vec) -> Result<()> { 102 | if let Some(reader) = node.as_any().downcast_ref::() { 103 | let schema: protobuf::Schema = reader.schema().try_into()?; 104 | let partitioning: protobuf::Partitioning = serialize_partitioning( 105 | reader.properties().output_partitioning(), 106 | &DefaultPhysicalExtensionCodec {}, 107 | )?; 108 | 109 | let pb = DfRayStageReaderExecNode { 110 | schema: Some(schema), 111 | partitioning: Some(partitioning), 112 | stage_id: reader.stage_id as u64, 113 | }; 114 | 115 | pb.encode(buf) 116 | .map_err(|e| internal_datafusion_err!("can't encode ray stage reader pb: {e}"))?; 117 | Ok(()) 118 | } else if let Some(pi) = node.as_any().downcast_ref::() { 119 | let pb = PartitionIsolatorExecNode { 120 | dummy: 0.0, 121 | partition_count: pi.partition_count as u64, 122 | }; 123 | 124 | pb.encode(buf) 125 | .map_err(|e| internal_datafusion_err!("can't encode partition isolator pb: {e}"))?; 126 | 127 | Ok(()) 128 | } else if let Some(max) = node.as_any().downcast_ref::() { 129 | let pb = MaxRowsExecNode { 130 | max_rows: max.max_rows as u64, 131 | }; 132 | pb.encode(buf) 133 | .map_err(|e| internal_datafusion_err!("can't encode max rows pb: {e}"))?; 134 | 135 | Ok(()) 136 | } else if let Some(pre) = node.as_any().downcast_ref::() { 137 | let pb = PrefetchExecNode { 138 | dummy: 0, 139 | buf_size: pre.buf_size as u64, 140 | }; 141 | pb.encode(buf) 142 | .map_err(|e| internal_datafusion_err!("can't encode prefetch pb: {e}"))?; 143 | 144 | Ok(()) 145 | } else { 146 | internal_err!("Not supported") 147 | } 148 | } 149 | } 150 | 151 | #[cfg(test)] 152 | mod test { 153 | use super::*; 154 | use crate::stage_reader::DFRayStageReaderExec; 155 | use arrow::datatypes::DataType; 156 | use datafusion::{ 157 | physical_plan::{Partitioning, display::DisplayableExecutionPlan, displayable}, 158 | prelude::SessionContext, 159 | }; 160 | use datafusion_proto::physical_plan::AsExecutionPlan; 161 | 162 | use std::sync::Arc; 163 | 164 | #[test] 165 | fn stage_reader_round_trip() { 166 | let schema = Arc::new(arrow::datatypes::Schema::new(vec![ 167 | arrow::datatypes::Field::new("a", DataType::Int32, false), 168 | arrow::datatypes::Field::new("b", DataType::Int32, false), 169 | ])); 170 | let ctx = SessionContext::new(); 171 | let part = Partitioning::UnknownPartitioning(2); 172 | let exec = Arc::new(DFRayStageReaderExec::try_new(part, schema, 1).unwrap()); 173 | let codec = RayCodec {}; 174 | let mut buf = vec![]; 175 | codec.try_encode(exec.clone(), &mut buf).unwrap(); 176 | let decoded = codec.try_decode(&buf, &[], &ctx).unwrap(); 177 | assert_eq!(exec.schema(), decoded.schema()); 178 | } 179 | #[test] 180 | fn max_rows_and_reader_round_trip() { 181 | let schema = Arc::new(arrow::datatypes::Schema::new(vec![ 182 | arrow::datatypes::Field::new("a", DataType::Int32, false), 183 | arrow::datatypes::Field::new("b", DataType::Int32, false), 184 | ])); 185 | let ctx = SessionContext::new(); 186 | let part = Partitioning::UnknownPartitioning(2); 187 | let exec = Arc::new(MaxRowsExec::new( 188 | Arc::new(DFRayStageReaderExec::try_new(part, schema, 1).unwrap()), 189 | 10, 190 | )); 191 | let codec = RayCodec {}; 192 | 193 | // serialize execution plan to proto 194 | let proto: protobuf::PhysicalPlanNode = 195 | protobuf::PhysicalPlanNode::try_from_physical_plan(exec.clone(), &codec) 196 | .expect("to proto"); 197 | 198 | // deserialize proto back to execution plan 199 | let runtime = ctx.runtime_env(); 200 | let result_exec_plan: Arc = proto 201 | .try_into_physical_plan(&ctx, runtime.as_ref(), &codec) 202 | .expect("from proto"); 203 | 204 | let input = displayable(exec.as_ref()).indent(true).to_string(); 205 | let round_trip = { 206 | let plan: &dyn ExecutionPlan = result_exec_plan.as_ref(); 207 | DisplayableExecutionPlan::new(plan) 208 | } 209 | .indent(true) 210 | .to_string(); 211 | assert_eq!(input, round_trip); 212 | } 213 | } 214 | -------------------------------------------------------------------------------- /src/context.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 datafusion::datasource::file_format::parquet::ParquetFormat; 19 | use datafusion::datasource::listing::{ListingOptions, ListingTableUrl}; 20 | use datafusion::execution::SessionStateBuilder; 21 | use datafusion::prelude::{CsvReadOptions, ParquetReadOptions, SessionConfig, SessionContext}; 22 | use datafusion_python::utils::wait_for_future; 23 | use log::debug; 24 | use pyo3::prelude::*; 25 | use std::sync::Arc; 26 | 27 | use crate::dataframe::DFRayDataFrame; 28 | use crate::physical::RayStageOptimizerRule; 29 | use crate::util::{maybe_register_object_store, ResultExt}; 30 | 31 | /// Internal Session Context object for the python class DFRayContext 32 | #[pyclass] 33 | pub struct DFRayContext { 34 | /// our datafusion context 35 | ctx: SessionContext, 36 | } 37 | 38 | #[pymethods] 39 | impl DFRayContext { 40 | #[new] 41 | pub fn new() -> PyResult { 42 | let rule = RayStageOptimizerRule::new(); 43 | 44 | let config = SessionConfig::default().with_information_schema(true); 45 | 46 | let state = SessionStateBuilder::new() 47 | .with_default_features() 48 | .with_physical_optimizer_rule(Arc::new(rule)) 49 | .with_config(config) 50 | .build(); 51 | 52 | let ctx = SessionContext::new_with_state(state); 53 | 54 | Ok(Self { ctx }) 55 | } 56 | 57 | pub fn register_parquet(&self, py: Python, name: String, path: String) -> PyResult<()> { 58 | let options = ParquetReadOptions::default(); 59 | 60 | let url = ListingTableUrl::parse(&path).to_py_err()?; 61 | 62 | maybe_register_object_store(&self.ctx, url.as_ref()).to_py_err()?; 63 | debug!("register_parquet: registering table {} at {}", name, path); 64 | 65 | wait_for_future(py, self.ctx.register_parquet(&name, &path, options.clone()))?; 66 | Ok(()) 67 | } 68 | 69 | pub fn register_csv(&self, py: Python, name: String, path: String) -> PyResult<()> { 70 | let options = CsvReadOptions::default(); 71 | 72 | let url = ListingTableUrl::parse(&path).to_py_err()?; 73 | 74 | maybe_register_object_store(&self.ctx, url.as_ref()).to_py_err()?; 75 | debug!("register_csv: registering table {} at {}", name, path); 76 | 77 | wait_for_future(py, self.ctx.register_csv(&name, &path, options.clone()))?; 78 | Ok(()) 79 | } 80 | 81 | #[pyo3(signature = (name, path, file_extension=".parquet"))] 82 | pub fn register_listing_table( 83 | &mut self, 84 | py: Python, 85 | name: &str, 86 | path: &str, 87 | file_extension: &str, 88 | ) -> PyResult<()> { 89 | let options = 90 | ListingOptions::new(Arc::new(ParquetFormat::new())).with_file_extension(file_extension); 91 | 92 | let path = format!("{path}/"); 93 | let url = ListingTableUrl::parse(&path).to_py_err()?; 94 | 95 | maybe_register_object_store(&self.ctx, url.as_ref()).to_py_err()?; 96 | 97 | debug!( 98 | "register_listing_table: registering table {} at {}", 99 | name, path 100 | ); 101 | wait_for_future( 102 | py, 103 | self.ctx 104 | .register_listing_table(name, path, options, None, None), 105 | ) 106 | .to_py_err() 107 | } 108 | 109 | pub fn sql(&self, py: Python, query: String) -> PyResult { 110 | let df = wait_for_future(py, self.ctx.sql(&query))?; 111 | 112 | Ok(DFRayDataFrame::new(df)) 113 | } 114 | 115 | pub fn set(&self, option: String, value: String) -> PyResult<()> { 116 | let state = self.ctx.state_ref(); 117 | let mut guard = state.write(); 118 | let config = guard.config_mut(); 119 | let options = config.options_mut(); 120 | options.set(&option, &value)?; 121 | 122 | Ok(()) 123 | } 124 | 125 | pub fn get_target_partitions(&self) -> usize { 126 | let state = self.ctx.state_ref(); 127 | let guard = state.read(); 128 | let config = guard.config(); 129 | let options = config.options(); 130 | options.execution.target_partitions 131 | } 132 | } 133 | -------------------------------------------------------------------------------- /src/flight.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::sync::Arc; 19 | 20 | use futures::stream::BoxStream; 21 | use tonic::{Request, Response, Status, Streaming}; 22 | 23 | use arrow_flight::{ 24 | flight_service_server::FlightService, Action, ActionType, Criteria, Empty, FlightData, 25 | FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse, PollInfo, PutResult, 26 | SchemaResult, Ticket, 27 | }; 28 | 29 | pub type DoGetStream = BoxStream<'static, Result>; 30 | 31 | #[tonic::async_trait] 32 | pub trait FlightHandler: Send + Sync { 33 | async fn get_stream(&self, request: Request) -> Result, Status>; 34 | } 35 | 36 | pub struct FlightServ { 37 | pub handler: Arc, 38 | } 39 | 40 | #[tonic::async_trait] 41 | impl FlightService for FlightServ { 42 | type HandshakeStream = BoxStream<'static, Result>; 43 | type ListFlightsStream = BoxStream<'static, Result>; 44 | type DoGetStream = BoxStream<'static, Result>; 45 | type DoPutStream = BoxStream<'static, Result>; 46 | type DoActionStream = BoxStream<'static, Result>; 47 | type ListActionsStream = BoxStream<'static, Result>; 48 | type DoExchangeStream = BoxStream<'static, Result>; 49 | 50 | async fn do_get( 51 | &self, 52 | request: Request, 53 | ) -> Result, Status> { 54 | self.handler.get_stream(request).await 55 | } 56 | 57 | async fn do_put( 58 | &self, 59 | _request: Request>, 60 | ) -> Result, Status> { 61 | Err(Status::unimplemented("Unimplemented: do put")) 62 | } 63 | 64 | async fn handshake( 65 | &self, 66 | _request: Request>, 67 | ) -> Result, Status> { 68 | Err(Status::unimplemented("Unimplemented: handshake")) 69 | } 70 | 71 | async fn list_flights( 72 | &self, 73 | _request: Request, 74 | ) -> Result, Status> { 75 | Err(Status::unimplemented("Unimplemented: list_flights")) 76 | } 77 | 78 | async fn get_flight_info( 79 | &self, 80 | _request: Request, 81 | ) -> Result, Status> { 82 | Err(Status::unimplemented("Unimplemented: get_flight_info")) 83 | } 84 | 85 | async fn poll_flight_info( 86 | &self, 87 | _request: Request, 88 | ) -> Result, Status> { 89 | Err(Status::unimplemented("Unimplemented: poll_flight_info")) 90 | } 91 | 92 | async fn get_schema( 93 | &self, 94 | _request: Request, 95 | ) -> Result, Status> { 96 | Err(Status::unimplemented("Unimplemented: get_schema")) 97 | } 98 | 99 | async fn do_action( 100 | &self, 101 | _request: Request, 102 | ) -> Result, Status> { 103 | Err(Status::unimplemented("Unimplemented: do action")) 104 | } 105 | 106 | async fn list_actions( 107 | &self, 108 | _request: Request, 109 | ) -> Result, Status> { 110 | Err(Status::unimplemented("Unimplemented: list_actions")) 111 | } 112 | 113 | async fn do_exchange( 114 | &self, 115 | _request: Request>, 116 | ) -> Result, Status> { 117 | Err(Status::unimplemented("Unimplemented: do_exchange")) 118 | } 119 | } 120 | -------------------------------------------------------------------------------- /src/isolator.rs: -------------------------------------------------------------------------------- 1 | use std::{fmt::Formatter, sync::Arc}; 2 | 3 | use datafusion::{ 4 | common::internal_datafusion_err, 5 | error::Result, 6 | execution::SendableRecordBatchStream, 7 | physical_plan::{ 8 | DisplayAs, DisplayFormatType, EmptyRecordBatchStream, ExecutionPlan, Partitioning, 9 | PlanProperties, 10 | }, 11 | }; 12 | use log::error; 13 | 14 | pub struct PartitionGroup(pub Vec); 15 | 16 | /// This is a simple execution plan that isolates a partition from the input plan 17 | /// It will advertise that it has a single partition and when 18 | /// asked to execute, it will execute a particular partition from the child 19 | /// input plan. 20 | /// 21 | /// This allows us to execute Repartition Exec's on different processes 22 | /// by showing each one only a single child partition 23 | #[derive(Debug)] 24 | pub struct PartitionIsolatorExec { 25 | pub input: Arc, 26 | properties: PlanProperties, 27 | pub partition_count: usize, 28 | } 29 | 30 | impl PartitionIsolatorExec { 31 | pub fn new(input: Arc, partition_count: usize) -> Self { 32 | // We advertise that we only have partition_count partitions 33 | let properties = input 34 | .properties() 35 | .clone() 36 | .with_partitioning(Partitioning::UnknownPartitioning(partition_count)); 37 | 38 | Self { 39 | input, 40 | properties, 41 | partition_count, 42 | } 43 | } 44 | } 45 | 46 | impl DisplayAs for PartitionIsolatorExec { 47 | fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { 48 | write!( 49 | f, 50 | "PartitionIsolatorExec [providing upto {} partitions]", 51 | self.partition_count 52 | ) 53 | } 54 | } 55 | 56 | impl ExecutionPlan for PartitionIsolatorExec { 57 | fn name(&self) -> &str { 58 | "PartitionIsolatorExec" 59 | } 60 | 61 | fn as_any(&self) -> &dyn std::any::Any { 62 | self 63 | } 64 | 65 | fn properties(&self) -> &PlanProperties { 66 | &self.properties 67 | } 68 | 69 | fn children(&self) -> Vec<&std::sync::Arc> { 70 | vec![&self.input] 71 | } 72 | 73 | fn with_new_children( 74 | self: std::sync::Arc, 75 | children: Vec>, 76 | ) -> Result> { 77 | // TODO: generalize this 78 | assert_eq!(children.len(), 1); 79 | Ok(Arc::new(Self::new( 80 | children[0].clone(), 81 | self.partition_count, 82 | ))) 83 | } 84 | 85 | fn execute( 86 | &self, 87 | partition: usize, 88 | context: std::sync::Arc, 89 | ) -> Result { 90 | let config = context.session_config(); 91 | let partition_group = &config 92 | .get_extension::() 93 | .ok_or(internal_datafusion_err!( 94 | "PartitionGroup not set in session config" 95 | ))? 96 | .0; 97 | 98 | if partition > self.partition_count { 99 | error!( 100 | "PartitionIsolatorExec asked to execute partition {} but only has {} partitions", 101 | partition, self.partition_count 102 | ); 103 | return Err(internal_datafusion_err!( 104 | "Invalid partition {} for PartitionIsolatorExec", 105 | partition 106 | )); 107 | } 108 | 109 | let output_stream = match partition_group.get(partition) { 110 | Some(actual_partition_number) => self.input.execute(*actual_partition_number, context), 111 | None => Ok(Box::pin(EmptyRecordBatchStream::new(self.input.schema())) 112 | as SendableRecordBatchStream), 113 | }; 114 | output_stream 115 | } 116 | } 117 | -------------------------------------------------------------------------------- /src/lib.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 | extern crate core; 19 | 20 | use pyo3::prelude::*; 21 | use std::env; 22 | 23 | mod proto; 24 | pub use proto::generated::protobuf; 25 | 26 | pub mod codec; 27 | pub mod context; 28 | pub mod dataframe; 29 | pub mod flight; 30 | pub mod isolator; 31 | pub mod max_rows; 32 | pub mod physical; 33 | pub mod pre_fetch; 34 | pub mod processor_service; 35 | pub mod stage; 36 | pub mod stage_reader; 37 | pub mod util; 38 | 39 | #[pymodule] 40 | fn _datafusion_ray_internal(m: &Bound<'_, PyModule>) -> PyResult<()> { 41 | setup_logging(); 42 | m.add_class::()?; 43 | m.add_class::()?; 44 | m.add_class::()?; 45 | m.add_class::()?; 46 | m.add_class::()?; 47 | m.add_function(wrap_pyfunction!(util::prettify, m)?)?; 48 | Ok(()) 49 | } 50 | 51 | fn setup_logging() { 52 | // ensure this python logger will route messages back to rust 53 | pyo3_pylogger::register("datafusion_ray"); 54 | 55 | let dfr_env = env::var("DATAFUSION_RAY_LOG_LEVEL").unwrap_or("WARN".to_string()); 56 | let rust_log_env = env::var("RUST_LOG").unwrap_or("WARN".to_string()); 57 | 58 | let combined_env = format!("{rust_log_env},datafusion_ray={dfr_env}"); 59 | 60 | env_logger::Builder::new() 61 | .parse_filters(&combined_env) 62 | .init(); 63 | } 64 | -------------------------------------------------------------------------------- /src/max_rows.rs: -------------------------------------------------------------------------------- 1 | use std::{fmt::Formatter, sync::Arc}; 2 | 3 | use datafusion::{ 4 | error::Result, 5 | execution::SendableRecordBatchStream, 6 | physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties}, 7 | }; 8 | 9 | use crate::util::max_rows_stream; 10 | 11 | /// An Execution plan that will not yield batches with greater than max_rows. 12 | /// 13 | /// If its input produces a batch with greater than max_rows it will zero-copy 14 | /// split the batch and continue to do this until the remaining batch has 15 | /// <= max_rows rows. It will yield each of these batches as separate Items 16 | #[derive(Debug)] 17 | pub struct MaxRowsExec { 18 | pub input: Arc, 19 | pub max_rows: usize, 20 | } 21 | 22 | impl MaxRowsExec { 23 | pub fn new(input: Arc, max_rows: usize) -> Self { 24 | Self { input, max_rows } 25 | } 26 | } 27 | 28 | impl DisplayAs for MaxRowsExec { 29 | fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { 30 | write!(f, "MaxRowsExec[max_rows={}]", self.max_rows) 31 | } 32 | } 33 | 34 | impl ExecutionPlan for MaxRowsExec { 35 | fn name(&self) -> &str { 36 | "MaxRowsExec" 37 | } 38 | 39 | fn as_any(&self) -> &dyn std::any::Any { 40 | self 41 | } 42 | 43 | fn properties(&self) -> &PlanProperties { 44 | self.input.properties() 45 | } 46 | 47 | fn children(&self) -> Vec<&std::sync::Arc> { 48 | vec![&self.input] 49 | } 50 | 51 | fn with_new_children( 52 | self: std::sync::Arc, 53 | children: Vec>, 54 | ) -> Result> { 55 | // TODO: generalize this 56 | assert_eq!(children.len(), 1); 57 | Ok(Arc::new(Self::new(children[0].clone(), self.max_rows))) 58 | } 59 | 60 | fn execute( 61 | &self, 62 | partition: usize, 63 | context: std::sync::Arc, 64 | ) -> Result { 65 | self.input 66 | .execute(partition, context) 67 | .map(|stream| max_rows_stream(stream, self.max_rows)) 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /src/physical.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 datafusion::common::tree_node::{Transformed, TreeNode}; 19 | use datafusion::error::Result; 20 | use datafusion::physical_optimizer::PhysicalOptimizerRule; 21 | use datafusion::physical_plan::joins::NestedLoopJoinExec; 22 | use datafusion::physical_plan::repartition::RepartitionExec; 23 | use datafusion::physical_plan::sorts::sort::SortExec; 24 | use datafusion::physical_plan::ExecutionPlan; 25 | use log::debug; 26 | use std::sync::Arc; 27 | 28 | use crate::stage::DFRayStageExec; 29 | use crate::util::display_plan_with_partition_counts; 30 | 31 | /// This optimizer rule walks up the physical plan tree 32 | /// and inserts RayStageExec nodes where appropriate to denote where we will split 33 | /// the plan into stages. 34 | /// 35 | /// The RayStageExec nodes are merely markers to inform where to break the plan up. 36 | /// 37 | /// Later, the plan will be examined again to actually split it up. 38 | /// These RayStageExecs serve as markers where we know to break it up on a network 39 | /// boundary and we can insert readers and writers as appropriate. 40 | #[derive(Debug)] 41 | pub struct RayStageOptimizerRule {} 42 | 43 | impl Default for RayStageOptimizerRule { 44 | fn default() -> Self { 45 | Self::new() 46 | } 47 | } 48 | 49 | impl RayStageOptimizerRule { 50 | pub fn new() -> Self { 51 | Self {} 52 | } 53 | } 54 | 55 | impl PhysicalOptimizerRule for RayStageOptimizerRule { 56 | fn optimize( 57 | &self, 58 | plan: Arc, 59 | _config: &datafusion::config::ConfigOptions, 60 | ) -> Result> { 61 | debug!( 62 | "optimizing physical plan:\n{}", 63 | display_plan_with_partition_counts(&plan) 64 | ); 65 | 66 | let mut stage_counter = 0; 67 | 68 | let up = |plan: Arc| { 69 | if plan.as_any().downcast_ref::().is_some() 70 | || plan.as_any().downcast_ref::().is_some() 71 | || plan.as_any().downcast_ref::().is_some() 72 | { 73 | let stage = Arc::new(DFRayStageExec::new(plan, stage_counter)); 74 | stage_counter += 1; 75 | Ok(Transformed::yes(stage as Arc)) 76 | } else { 77 | Ok(Transformed::no(plan)) 78 | } 79 | }; 80 | 81 | let plan = plan.transform_up(up)?.data; 82 | let final_plan = 83 | Arc::new(DFRayStageExec::new(plan, stage_counter)) as Arc; 84 | 85 | debug!( 86 | "optimized physical plan:\n{}", 87 | display_plan_with_partition_counts(&final_plan) 88 | ); 89 | Ok(final_plan) 90 | } 91 | 92 | fn name(&self) -> &str { 93 | "RayStageOptimizerRule" 94 | } 95 | 96 | fn schema_check(&self) -> bool { 97 | true 98 | } 99 | } 100 | -------------------------------------------------------------------------------- /src/pre_fetch.rs: -------------------------------------------------------------------------------- 1 | use std::{fmt::Formatter, sync::Arc}; 2 | 3 | use datafusion::error::Result; 4 | use datafusion::physical_plan::stream::RecordBatchStreamAdapter; 5 | use datafusion::physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties}; 6 | use datafusion::{arrow::datatypes::SchemaRef, execution::SendableRecordBatchStream}; 7 | use futures::stream::StreamExt; 8 | use tokio::sync::mpsc::channel; 9 | 10 | /// An execution plan that will try to consume and buffer RecordBatches from its input. 11 | /// It will hold those buffers in a bounded channel and serve them from the channel requested 12 | /// through execute(). 13 | /// 14 | /// The buffering begins when execute() is called. 15 | #[derive(Debug)] 16 | pub struct PrefetchExec { 17 | /// Input plan 18 | pub(crate) input: Arc, 19 | /// maximum amount of buffered RecordBatches 20 | pub(crate) buf_size: usize, 21 | /// our plan Properties, the same as our input 22 | properties: PlanProperties, 23 | } 24 | 25 | impl PrefetchExec { 26 | pub fn new(input: Arc, buf_size: usize) -> Self { 27 | // check for only one input 28 | if input.children().len() != 1 { 29 | panic!("PrefetchExec must have exactly one input"); 30 | } 31 | let properties = input.children()[0].properties().clone(); 32 | Self { 33 | input, 34 | buf_size, 35 | properties, 36 | } 37 | } 38 | } 39 | impl DisplayAs for PrefetchExec { 40 | fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { 41 | write!(f, "PrefetchExec [num={}]", self.buf_size) 42 | } 43 | } 44 | 45 | impl ExecutionPlan for PrefetchExec { 46 | fn schema(&self) -> SchemaRef { 47 | self.input.schema() 48 | } 49 | fn children(&self) -> Vec<&Arc> { 50 | vec![&self.input] 51 | } 52 | 53 | fn name(&self) -> &str { 54 | "PrefetchExec" 55 | } 56 | 57 | fn as_any(&self) -> &dyn std::any::Any { 58 | self 59 | } 60 | 61 | fn properties(&self) -> &datafusion::physical_plan::PlanProperties { 62 | &self.properties 63 | } 64 | 65 | fn with_new_children( 66 | self: std::sync::Arc, 67 | children: Vec>, 68 | ) -> datafusion::error::Result> { 69 | // TODO: handle more general case 70 | assert_eq!(children.len(), 1); 71 | let child = children[0].clone(); 72 | Ok(Arc::new(PrefetchExec::new(child, self.buf_size))) 73 | } 74 | 75 | fn execute( 76 | &self, 77 | partition: usize, 78 | context: std::sync::Arc, 79 | ) -> Result { 80 | let (tx, mut rx) = channel(self.buf_size); 81 | 82 | let mut input_stream = self.input.execute(partition, context)?; 83 | 84 | let consume_fut = async move { 85 | while let Some(batch) = input_stream.next().await { 86 | // TODO: how to neatly errors within this macro? 87 | tx.send(batch).await.unwrap(); 88 | } 89 | }; 90 | 91 | tokio::spawn(consume_fut); 92 | 93 | let out_stream = async_stream::stream! { 94 | while let Some(batch) = rx.recv().await { 95 | yield batch; 96 | } 97 | }; 98 | 99 | Ok(Box::pin(RecordBatchStreamAdapter::new( 100 | self.schema().clone(), 101 | out_stream, 102 | ))) 103 | } 104 | } 105 | -------------------------------------------------------------------------------- /src/proto/datafusion_ray.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | 3 | package datafusion_ray.protobuf; 4 | 5 | option java_multiple_files = true; 6 | option java_package = "datafusion_ray.protobuf"; 7 | option java_outer_classname = "RayDataFusionProto"; 8 | 9 | import "datafusion_common.proto"; 10 | import "datafusion.proto"; 11 | 12 | message DFRayStageReaderExecNode { 13 | // schema of the stage we will consume 14 | datafusion_common.Schema schema = 1; 15 | // properties of the stage we will consume 16 | datafusion.Partitioning partitioning = 2; 17 | // stage to read from 18 | uint64 stage_id = 3; 19 | } 20 | 21 | // the simplicity of the decoder in src/codec.rs currently requires a different byte 22 | // representation per message. Hence the dummy fields. 23 | // 24 | // I'll come back to this and sort it out. Its not super critical as the plans are 25 | // only exchanged at the start of queries, not during execution. 26 | 27 | message MaxRowsExecNode { 28 | uint64 max_rows = 1; 29 | } 30 | 31 | message PrefetchExecNode { 32 | uint32 dummy = 1; 33 | uint64 buf_size = 2; 34 | } 35 | 36 | message PartitionIsolatorExecNode { 37 | float dummy = 1; 38 | uint64 partition_count = 2; 39 | } 40 | 41 | // TODO: why, if FlightTicketData has the uint64 field first can it also be decoded also 42 | // MaxRowsExecNode? There is something I don't understand here 43 | message FlightTicketData { 44 | // stage id of the stream 45 | // parittion id of the stream 46 | bool dummy = 1; 47 | uint64 partition = 2; 48 | } 49 | 50 | -------------------------------------------------------------------------------- /src/proto/generated/mod.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 | pub mod protobuf; 19 | -------------------------------------------------------------------------------- /src/proto/generated/protobuf.rs: -------------------------------------------------------------------------------- 1 | #[allow(clippy::derive_partial_eq_without_eq)] 2 | #[derive(Clone, PartialEq, ::prost::Message)] 3 | pub struct DfRayStageReaderExecNode { 4 | /// schema of the stage we will consume 5 | #[prost(message, optional, tag = "1")] 6 | pub schema: ::core::option::Option<::datafusion_proto::protobuf::Schema>, 7 | /// properties of the stage we will consume 8 | #[prost(message, optional, tag = "2")] 9 | pub partitioning: ::core::option::Option<::datafusion_proto::protobuf::Partitioning>, 10 | /// stage to read from 11 | #[prost(uint64, tag = "3")] 12 | pub stage_id: u64, 13 | } 14 | #[allow(clippy::derive_partial_eq_without_eq)] 15 | #[derive(Clone, PartialEq, ::prost::Message)] 16 | pub struct MaxRowsExecNode { 17 | #[prost(uint64, tag = "1")] 18 | pub max_rows: u64, 19 | } 20 | #[allow(clippy::derive_partial_eq_without_eq)] 21 | #[derive(Clone, PartialEq, ::prost::Message)] 22 | pub struct PrefetchExecNode { 23 | #[prost(uint32, tag = "1")] 24 | pub dummy: u32, 25 | #[prost(uint64, tag = "2")] 26 | pub buf_size: u64, 27 | } 28 | #[allow(clippy::derive_partial_eq_without_eq)] 29 | #[derive(Clone, PartialEq, ::prost::Message)] 30 | pub struct PartitionIsolatorExecNode { 31 | #[prost(float, tag = "1")] 32 | pub dummy: f32, 33 | #[prost(uint64, tag = "2")] 34 | pub partition_count: u64, 35 | } 36 | /// TODO: why, if FlightTicketData has the uint64 field first can it also be decoded also 37 | /// MaxRowsExecNode? There is something I don't understand here 38 | #[allow(clippy::derive_partial_eq_without_eq)] 39 | #[derive(Clone, PartialEq, ::prost::Message)] 40 | pub struct FlightTicketData { 41 | /// stage id of the stream 42 | /// parittion id of the stream 43 | #[prost(bool, tag = "1")] 44 | pub dummy: bool, 45 | #[prost(uint64, tag = "2")] 46 | pub partition: u64, 47 | } 48 | -------------------------------------------------------------------------------- /src/proto/mod.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 | #[rustfmt::skip] 19 | pub mod generated; 20 | -------------------------------------------------------------------------------- /src/stage.rs: -------------------------------------------------------------------------------- 1 | use std::{fmt::Formatter, sync::Arc}; 2 | 3 | use datafusion::error::Result; 4 | use datafusion::physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties}; 5 | use datafusion::{arrow::datatypes::SchemaRef, execution::SendableRecordBatchStream}; 6 | 7 | /// An execution plan that serves as a marker of where we want to split the physical plan into 8 | /// stages. 9 | /// 10 | /// This marker is consumed later by the [`crate::dataframe::RayDataFrame`], when we are told to execute. It will 11 | /// create the discrete stages and insert other ExecutionPlans to read and write the data 12 | /// 13 | /// # Example 14 | /// The following query, 15 | /// ```sql 16 | /// select c.c_name, sum(o.o_totalprice) as total 17 | /// from orders o inner join customer c on o.o_c ustkey = c.c_custkey 18 | /// group by c_name limit 1 19 | /// ``` 20 | /// 21 | /// Will produce the following physical_plan from the optimizer 22 | /// 23 | /// ` 24 | /// RayStageExec[3] (output_partitioning=UnknownPartitioning(1)) 25 | /// ProjectionExec: expr=[c_name@0 as c_name, sum(o.o_totalprice)@1 as total] 26 | /// GlobalLimitExec: skip=0, fetch=1 27 | /// CoalescePartitionsExec 28 | /// AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name], aggr=[sum(o.o_totalprice)] 29 | /// RayStageExec[2] (output_partitioning=Hash([Column { name: "c_name", index: 0 }], 2)) 30 | /// RepartitionExec: partitioning=Hash([c_name@0], 2), input_partitions=2 31 | /// AggregateExec: mode=Partial, gby=[c_name@1 as c_name], aggr=[sum(o.o_totalprice)] 32 | /// ProjectionExec: expr=[o_totalprice@1 as o_totalprice, c_name@0 as c_name] 33 | /// HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@0)], projection=[c_name@1, o_totalprice@3] 34 | /// RayStageExec[0] (output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 2)) 35 | /// RepartitionExec: partitioning=Hash([c_custkey@0], 2), input_partitions=1 36 | /// ParquetExec: file_groups={1 group: [[.../customer.parquet]]}, projection=[c_custkey, c_name] 37 | /// RayStageExec[1] (output_partitioning=Hash([Column { name: "o_custkey", index: 0 }], 2)) 38 | /// RepartitionExec: partitioning=Hash([o_custkey@0], 2), input_partitions=2 39 | /// ParquetExec: file_groups={2 groups: [[.../orders.parquet:0..19037604], [.../orders.parquet:19037604..38075207]]}, projection=[o_custkey, o_totalprice] 40 | /// ` 41 | /// This physical plan will be split into 4 stages, as indicated by the RayStageExec nodes. Those 42 | /// stages will look like this: 43 | /// 44 | /// ` 45 | /// Stage 0 output partitions:2 shadow partitions: 1 46 | /// MaxRowsExec[max_rows=8192] 47 | /// CoalesceBatchesExec: target_batch_size=8192 48 | /// RepartitionExec: partitioning=Hash([c_custkey@0], 2), input_partitions=1 49 | /// PartitionIsolatorExec 50 | /// ParquetExec: file_groups={1 group: [[.../customer.parquet]]}, projection=[c_custkey, c_name] 51 | /// 52 | /// Stage 1 output partitions:2 shadow partitions: 2 53 | /// MaxRowsExec[max_rows=8192] 54 | /// CoalesceBatchesExec: target_batch_size=8192 55 | /// RepartitionExec: partitioning=Hash([o_custkey@0], 2), input_partitions=1 56 | /// PartitionIsolatorExec 57 | /// ParquetExec: file_groups={2 groups: [[.../orders.parquet:0..19037604], [.../orders.parquet:19037604..38075207]]}, projection=[o_custkey, o_totalprice] 58 | /// 59 | /// Stage 2 output partitions:2 shadow partitions: 2 60 | /// MaxRowsExec[max_rows=8192] 61 | /// CoalesceBatchesExec: target_batch_size=8192 62 | /// RepartitionExec: partitioning=Hash([c_name@0], 2), input_partitions=1 63 | /// PartitionIsolatorExec 64 | /// AggregateExec: mode=Partial, gby=[c_name@1 as c_name], aggr=[sum(o.o_totalprice)] 65 | /// ProjectionExec: expr=[o_totalprice@1 as o_totalprice, c_name@0 as c_name] 66 | /// HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@0)], projection=[c_name@1, o_totalprice@3] 67 | /// RayStageReaderExec[0] (output_partitioning=UnknownPartitioning(2)) 68 | /// RayStageReaderExec[1] (output_partitioning=UnknownPartitioning(2)) 69 | /// 70 | /// Stage 3 output partitions:1 shadow partitions: None 71 | /// MaxRowsExec[max_rows=8192] 72 | /// CoalesceBatchesExec: target_batch_size=8192 73 | /// ProjectionExec: expr=[c_name@0 as c_name, sum(o.o_totalprice)@1 as total] 74 | /// GlobalLimitExec: skip=0, fetch=1 75 | /// CoalescePartitionsExec 76 | /// AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name], aggr=[sum(o.o_totalprice)] 77 | /// RayStageReaderExec[2] (output_partitioning=UnknownPartitioning(2)) 78 | /// ` 79 | /// 80 | /// See [`crate::isolator::PartitionIsolatorExec`] for more information on how the shadow partitions work 81 | #[derive(Debug)] 82 | pub struct DFRayStageExec { 83 | /// Input plan 84 | pub(crate) input: Arc, 85 | /// Output partitioning 86 | properties: PlanProperties, 87 | pub stage_id: usize, 88 | } 89 | 90 | impl DFRayStageExec { 91 | pub fn new(input: Arc, stage_id: usize) -> Self { 92 | let properties = input.properties().clone(); 93 | 94 | Self { 95 | input, 96 | properties, 97 | stage_id, 98 | } 99 | } 100 | 101 | fn new_with_properties( 102 | input: Arc, 103 | stage_id: usize, 104 | properties: PlanProperties, 105 | ) -> Self { 106 | Self { 107 | input, 108 | properties, 109 | stage_id, 110 | } 111 | } 112 | } 113 | impl DisplayAs for DFRayStageExec { 114 | fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { 115 | write!( 116 | f, 117 | "RayStageExec[{}] (output_partitioning={:?})", 118 | self.stage_id, 119 | self.properties().partitioning 120 | ) 121 | } 122 | } 123 | 124 | impl ExecutionPlan for DFRayStageExec { 125 | fn schema(&self) -> SchemaRef { 126 | self.input.schema() 127 | } 128 | fn children(&self) -> Vec<&Arc> { 129 | vec![&self.input] 130 | } 131 | 132 | fn name(&self) -> &str { 133 | "RayStageExec" 134 | } 135 | 136 | fn as_any(&self) -> &dyn std::any::Any { 137 | self 138 | } 139 | 140 | fn properties(&self) -> &datafusion::physical_plan::PlanProperties { 141 | &self.properties 142 | } 143 | 144 | fn with_new_children( 145 | self: std::sync::Arc, 146 | children: Vec>, 147 | ) -> datafusion::error::Result> { 148 | // TODO: handle more general case 149 | assert_eq!(children.len(), 1); 150 | let child = children[0].clone(); 151 | 152 | // as the plan tree is rearranged we want to remember the original partitioning that we 153 | // had, even if we get new inputs. This is because RayStageReaderExecs, when created by 154 | // the RayDataFrame will need to know the original partitioning 155 | Ok(Arc::new(DFRayStageExec::new_with_properties( 156 | child, 157 | self.stage_id, 158 | self.properties.clone(), 159 | ))) 160 | } 161 | 162 | /// We will have to defer this functionality to python as Ray does not yet have Rust bindings. 163 | fn execute( 164 | &self, 165 | _partition: usize, 166 | _context: std::sync::Arc, 167 | ) -> Result { 168 | unimplemented!("Ray Stage Exec") 169 | } 170 | } 171 | -------------------------------------------------------------------------------- /src/stage_reader.rs: -------------------------------------------------------------------------------- 1 | use std::{fmt::Formatter, sync::Arc}; 2 | 3 | use arrow_flight::{FlightClient, Ticket}; 4 | use datafusion::common::{internal_datafusion_err, internal_err}; 5 | use datafusion::error::Result; 6 | use datafusion::physical_expr::EquivalenceProperties; 7 | use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; 8 | use datafusion::physical_plan::stream::RecordBatchStreamAdapter; 9 | use datafusion::physical_plan::{ 10 | DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, 11 | }; 12 | use datafusion::{arrow::datatypes::SchemaRef, execution::SendableRecordBatchStream}; 13 | use futures::stream::TryStreamExt; 14 | use futures::StreamExt; 15 | use log::trace; 16 | use prost::Message; 17 | 18 | use crate::processor_service::ServiceClients; 19 | use crate::protobuf::FlightTicketData; 20 | use crate::util::CombinedRecordBatchStream; 21 | 22 | /// An [`ExecutionPlan`] that will produce a stream of batches fetched from another stage 23 | /// which is hosted by a [`crate::stage_service::StageService`] separated from a network boundary 24 | /// 25 | /// Note that discovery of the service is handled by populating an instance of [`crate::stage_service::ServiceClients`] 26 | /// and storing it as an extension in the [`datafusion::execution::TaskContext`] configuration. 27 | #[derive(Debug)] 28 | pub struct DFRayStageReaderExec { 29 | properties: PlanProperties, 30 | schema: SchemaRef, 31 | pub stage_id: usize, 32 | } 33 | 34 | impl DFRayStageReaderExec { 35 | pub fn try_new_from_input(input: Arc, stage_id: usize) -> Result { 36 | let properties = input.properties().clone(); 37 | 38 | Self::try_new(properties.partitioning.clone(), input.schema(), stage_id) 39 | } 40 | 41 | pub fn try_new(partitioning: Partitioning, schema: SchemaRef, stage_id: usize) -> Result { 42 | let properties = PlanProperties::new( 43 | EquivalenceProperties::new(schema.clone()), 44 | Partitioning::UnknownPartitioning(partitioning.partition_count()), 45 | EmissionType::Incremental, 46 | Boundedness::Bounded, 47 | ); 48 | 49 | Ok(Self { 50 | properties, 51 | schema, 52 | stage_id, 53 | }) 54 | } 55 | } 56 | impl DisplayAs for DFRayStageReaderExec { 57 | fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { 58 | write!( 59 | f, 60 | "RayStageReaderExec[{}] (output_partitioning={:?})", 61 | self.stage_id, 62 | self.properties().partitioning 63 | ) 64 | } 65 | } 66 | 67 | impl ExecutionPlan for DFRayStageReaderExec { 68 | fn schema(&self) -> SchemaRef { 69 | self.schema.clone() 70 | } 71 | fn children(&self) -> Vec<&Arc> { 72 | vec![] 73 | } 74 | 75 | fn name(&self) -> &str { 76 | "RayStageReaderExec" 77 | } 78 | 79 | fn as_any(&self) -> &dyn std::any::Any { 80 | self 81 | } 82 | 83 | fn properties(&self) -> &datafusion::physical_plan::PlanProperties { 84 | &self.properties 85 | } 86 | 87 | fn with_new_children( 88 | self: std::sync::Arc, 89 | _children: Vec>, 90 | ) -> datafusion::error::Result> { 91 | // TODO: handle more general case 92 | unimplemented!() 93 | } 94 | 95 | fn execute( 96 | &self, 97 | partition: usize, 98 | context: std::sync::Arc, 99 | ) -> Result { 100 | let name = format!("RayStageReaderExec[{}-{}]:", self.stage_id, partition); 101 | trace!("{name} execute"); 102 | let client_map = &context 103 | .session_config() 104 | .get_extension::() 105 | .ok_or(internal_datafusion_err!( 106 | "{name} Flight Client not in context" 107 | ))? 108 | .clone() 109 | .0; 110 | 111 | trace!("{name} client_map keys {:?}", client_map.keys()); 112 | 113 | let clients = client_map 114 | .get(&(self.stage_id, partition)) 115 | .ok_or(internal_datafusion_err!( 116 | "{} No flight clients found for {}:{}, have {:?}", 117 | name, 118 | self.stage_id, 119 | partition, 120 | client_map.keys() 121 | ))? 122 | .lock() 123 | .iter() 124 | .map(|c| { 125 | let inner_clone = c.inner().clone(); 126 | FlightClient::new_from_inner(inner_clone) 127 | }) 128 | .collect::>(); 129 | 130 | let ftd = FlightTicketData { 131 | dummy: false, 132 | partition: partition as u64, 133 | }; 134 | 135 | let ticket = Ticket { 136 | ticket: ftd.encode_to_vec().into(), 137 | }; 138 | 139 | let schema = self.schema.clone(); 140 | 141 | let stream = async_stream::stream! { 142 | let mut error = false; 143 | 144 | let mut streams = vec![]; 145 | for mut client in clients { 146 | let name = name.clone(); 147 | trace!("{name} Getting flight stream" ); 148 | match client.do_get(ticket.clone()).await { 149 | Ok(flight_stream) => { 150 | trace!("{name} Got flight stream. headers:{:?}", flight_stream.headers()); 151 | let rbr_stream = RecordBatchStreamAdapter::new(schema.clone(), 152 | flight_stream 153 | .map_err(move |e| internal_datafusion_err!("{} Error consuming flight stream: {}", name, e))); 154 | 155 | streams.push(Box::pin(rbr_stream) as SendableRecordBatchStream); 156 | }, 157 | Err(e) => { 158 | error = true; 159 | yield internal_err!("{} Error getting flight stream: {}", name, e); 160 | } 161 | } 162 | } 163 | if !error { 164 | let mut combined = CombinedRecordBatchStream::new(schema.clone(),streams); 165 | 166 | while let Some(maybe_batch) = combined.next().await { 167 | yield maybe_batch; 168 | } 169 | } 170 | 171 | }; 172 | 173 | Ok(Box::pin(RecordBatchStreamAdapter::new( 174 | self.schema.clone(), 175 | stream, 176 | ))) 177 | } 178 | } 179 | -------------------------------------------------------------------------------- /testdata/tips/tips.parquet: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/datafusion-ray/1c63e854cd9ec2c19703d03399704fb8ece04b20/testdata/tips/tips.parquet -------------------------------------------------------------------------------- /testdata/tpch/.gitignore: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/datafusion-ray/1c63e854cd9ec2c19703d03399704fb8ece04b20/testdata/tpch/.gitignore -------------------------------------------------------------------------------- /tpch/make_data.py: -------------------------------------------------------------------------------- 1 | import duckdb 2 | 3 | import sys 4 | 5 | conn = duckdb.connect() 6 | 7 | 8 | def make(scale_factor: int, output_path: str): 9 | statements = [ 10 | "install tpch", 11 | "load tpch", 12 | f"call dbgen(sf = {scale_factor})", 13 | ] 14 | execute(statements) 15 | 16 | statements = [] 17 | for row in conn.execute("show tables").fetchall(): 18 | table = row[0] 19 | statements.append( 20 | f"copy {table} to '{output_path}/{table}.parquet' (format parquet, compression zstd)" 21 | ) 22 | execute(statements) 23 | 24 | 25 | def execute(statements): 26 | for statement in statements: 27 | print(f"executing: {statement}") 28 | conn.execute(statement) 29 | 30 | 31 | if __name__ == "__main__": 32 | make(int(sys.argv[1]), sys.argv[2]) 33 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/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 | -------------------------------------------------------------------------------- /tpch/requirements.txt: -------------------------------------------------------------------------------- 1 | duckdb 2 | ray[default] 3 | maturin 4 | -------------------------------------------------------------------------------- /tpch/tpcbench.py: -------------------------------------------------------------------------------- 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 | import argparse 19 | import ray 20 | from datafusion_ray import DFRayContext, df_ray_runtime_env 21 | from datafusion_ray.util import LocalValidator, prettify 22 | from datetime import datetime 23 | import json 24 | import os 25 | import time 26 | 27 | 28 | def tpch_query(qnum: int) -> str: 29 | query_path = os.path.join(os.path.dirname(__file__), "queries") 30 | return open(os.path.join(query_path, f"q{qnum}.sql")).read() 31 | 32 | 33 | def main( 34 | qnum: int, 35 | data_path: str, 36 | concurrency: int, 37 | batch_size: int, 38 | partitions_per_processor: int | None, 39 | processor_pool_min: int, 40 | listing_tables: bool, 41 | validate: bool, 42 | output_path: str, 43 | prefetch_buffer_size: int, 44 | ): 45 | # Register the tables 46 | table_names = [ 47 | "customer", 48 | "lineitem", 49 | "nation", 50 | "orders", 51 | "part", 52 | "partsupp", 53 | "region", 54 | "supplier", 55 | ] 56 | # Connect to a cluster 57 | # use ray job submit 58 | ray.init(runtime_env=df_ray_runtime_env) 59 | 60 | ctx = DFRayContext( 61 | batch_size=batch_size, 62 | partitions_per_processor=partitions_per_processor, 63 | prefetch_buffer_size=prefetch_buffer_size, 64 | processor_pool_min=processor_pool_min, 65 | processor_pool_max=1000, 66 | ) 67 | 68 | local = LocalValidator() 69 | 70 | ctx.set("datafusion.execution.target_partitions", f"{concurrency}") 71 | # ctx.set("datafusion.execution.parquet.pushdown_filters", "true") 72 | ctx.set("datafusion.optimizer.enable_round_robin_repartition", "false") 73 | ctx.set("datafusion.execution.coalesce_batches", "false") 74 | 75 | for table in table_names: 76 | path = os.path.join(data_path, f"{table}.parquet") 77 | print(f"Registering table {table} using path {path}") 78 | if listing_tables: 79 | ctx.register_listing_table(table, path) 80 | local.register_listing_table(table, path) 81 | else: 82 | ctx.register_parquet(table, path) 83 | local.register_parquet(table, path) 84 | 85 | current_time_millis = int(datetime.now().timestamp() * 1000) 86 | results_path = os.path.join( 87 | output_path, f"datafusion-ray-tpch-{current_time_millis}.json" 88 | ) 89 | print(f"Writing results to {results_path}") 90 | 91 | results = { 92 | "engine": "datafusion-ray", 93 | "benchmark": "tpch", 94 | "settings": { 95 | "concurrency": concurrency, 96 | "batch_size": batch_size, 97 | "prefetch_buffer_size": prefetch_buffer_size, 98 | "partitions_per_processor": partitions_per_processor, 99 | }, 100 | "data_path": data_path, 101 | "queries": {}, 102 | } 103 | if validate: 104 | results["validated"] = {} 105 | 106 | queries = range(1, 23) if qnum == -1 else [qnum] 107 | for qnum in queries: 108 | sql = tpch_query(qnum) 109 | 110 | statements = list( 111 | filter( 112 | lambda x: len(x) > 0, map(lambda x: x.strip(), sql.split(";")) 113 | ) 114 | ) 115 | 116 | start_time = time.time() 117 | all_batches = [] 118 | for sql in statements: 119 | print("executing ", sql) 120 | df = ctx.sql(sql) 121 | all_batches.append(df.collect()) 122 | end_time = time.time() 123 | results["queries"][qnum] = end_time - start_time 124 | 125 | calculated = "\n".join([prettify(b) for b in all_batches]) 126 | print(calculated) 127 | out_path = os.path.join( 128 | output_path, f"datafusion_ray_tpch_q{qnum}_result.txt" 129 | ) 130 | with open(out_path, "w") as f: 131 | f.write(calculated) 132 | 133 | if validate: 134 | all_batches = [] 135 | for sql in statements: 136 | all_batches.append(local.collect_sql(sql)) 137 | expected = "\n".join([prettify(b) for b in all_batches]) 138 | 139 | results["validated"][qnum] = calculated == expected 140 | print(f"done with query {qnum}") 141 | 142 | # write the results as we go, so you can peek at them 143 | results_dump = json.dumps(results, indent=4) 144 | with open(results_path, "w+") as f: 145 | f.write(results_dump) 146 | 147 | # write results to stdout 148 | print(results_dump) 149 | 150 | # give ray a moment to clean up 151 | print("benchmark complete. sleeping for 3 seconds for ray to clean up") 152 | time.sleep(3) 153 | 154 | if validate and False in results["validated"].values(): 155 | # return a non zero return code if we did not validate all queries 156 | print("Possible incorrect query result") 157 | exit(1) 158 | 159 | 160 | if __name__ == "__main__": 161 | parser = argparse.ArgumentParser( 162 | description="DataFusion benchmark derived from TPC-H / TPC-DS" 163 | ) 164 | parser.add_argument("--data", required=True, help="Path to data files") 165 | parser.add_argument( 166 | "--concurrency", required=True, help="Number of concurrent tasks" 167 | ) 168 | parser.add_argument( 169 | "--qnum", type=int, default=-1, help="TPCH query number, 1-22" 170 | ) 171 | parser.add_argument("--listing-tables", action="store_true") 172 | parser.add_argument("--validate", action="store_true") 173 | parser.add_argument( 174 | "--log-level", default="INFO", help="ERROR,WARN,INFO,DEBUG,TRACE" 175 | ) 176 | parser.add_argument( 177 | "--batch-size", 178 | required=False, 179 | default=8192, 180 | help="Desired batch size output per stage", 181 | ) 182 | parser.add_argument( 183 | "--partitions-per-processor", 184 | type=int, 185 | help="partitions per DFRayProcessor", 186 | ) 187 | parser.add_argument( 188 | "--output-path", 189 | type=str, 190 | default=".", 191 | help="directory to write output json", 192 | ) 193 | 194 | parser.add_argument( 195 | "--prefetch-buffer-size", 196 | required=False, 197 | default=0, 198 | type=int, 199 | help="How many batches each stage should eagerly buffer", 200 | ) 201 | parser.add_argument( 202 | "--processor-pool-min", 203 | type=int, 204 | help="Minimum number of DFRayProcessors to keep in pool", 205 | ) 206 | 207 | args = parser.parse_args() 208 | 209 | main( 210 | args.qnum, 211 | args.data, 212 | int(args.concurrency), 213 | int(args.batch_size), 214 | args.partitions_per_processor, 215 | args.processor_pool_min, 216 | args.listing_tables, 217 | args.validate, 218 | args.output_path, 219 | args.prefetch_buffer_size, 220 | ) 221 | --------------------------------------------------------------------------------