Showing preview only (1,357K chars total). Download the full file or copy to clipboard to get everything.
Repository: nils-braun/dask-sql
Branch: main
Commit: 775b56fb8f99
Files: 251
Total size: 1.3 MB
Directory structure:
gitextract_q18bbzy0/
├── .cargo/
│ └── config.toml
├── .coveragerc
├── .dockerignore
├── .github/
│ ├── CODEOWNERS
│ ├── ISSUE_TEMPLATE/
│ │ ├── bug_report.md
│ │ ├── documentation-request.md
│ │ ├── feature_request.md
│ │ └── submit-question.md
│ ├── dependabot.yml
│ └── workflows/
│ ├── conda.yml
│ ├── docker.yml
│ ├── release.yml
│ ├── rust.yml
│ ├── style.yml
│ ├── test-upstream.yml
│ └── test.yml
├── .gitignore
├── .pre-commit-config.yaml
├── .readthedocs.yaml
├── CODE_OF_CONDUCT.md
├── CONTRIBUTING.md
├── Cargo.toml
├── LICENSE.txt
├── MANIFEST.in
├── README.md
├── conftest.py
├── continuous_integration/
│ ├── docker/
│ │ ├── cloud.dockerfile
│ │ ├── conda.txt
│ │ └── main.dockerfile
│ ├── environment-3.10.yaml
│ ├── environment-3.11.yaml
│ ├── environment-3.12.yaml
│ ├── environment-3.9.yaml
│ ├── gpuci/
│ │ ├── environment-3.10.yaml
│ │ ├── environment-3.11.yaml
│ │ └── environment-3.9.yaml
│ ├── recipe/
│ │ ├── build.sh
│ │ ├── conda_build_config.yaml
│ │ ├── meta.yaml
│ │ └── run_test.py
│ └── scripts/
│ ├── startup_script.py
│ └── update-dependencies.sh
├── dask_sql/
│ ├── __init__.py
│ ├── _compat.py
│ ├── cmd.py
│ ├── config.py
│ ├── context.py
│ ├── datacontainer.py
│ ├── input_utils/
│ │ ├── __init__.py
│ │ ├── base.py
│ │ ├── convert.py
│ │ ├── dask.py
│ │ ├── hive.py
│ │ ├── intake.py
│ │ ├── location.py
│ │ ├── pandaslike.py
│ │ └── sqlalchemy.py
│ ├── integrations/
│ │ ├── __init__.py
│ │ ├── fugue.py
│ │ └── ipython.py
│ ├── mappings.py
│ ├── physical/
│ │ ├── __init__.py
│ │ ├── rel/
│ │ │ ├── __init__.py
│ │ │ ├── base.py
│ │ │ ├── convert.py
│ │ │ ├── custom/
│ │ │ │ ├── __init__.py
│ │ │ │ ├── alter.py
│ │ │ │ ├── analyze_table.py
│ │ │ │ ├── create_catalog_schema.py
│ │ │ │ ├── create_experiment.py
│ │ │ │ ├── create_memory_table.py
│ │ │ │ ├── create_model.py
│ │ │ │ ├── create_table.py
│ │ │ │ ├── describe_model.py
│ │ │ │ ├── distributeby.py
│ │ │ │ ├── drop_model.py
│ │ │ │ ├── drop_schema.py
│ │ │ │ ├── drop_table.py
│ │ │ │ ├── export_model.py
│ │ │ │ ├── metrics.py
│ │ │ │ ├── predict_model.py
│ │ │ │ ├── show_columns.py
│ │ │ │ ├── show_models.py
│ │ │ │ ├── show_schemas.py
│ │ │ │ ├── show_tables.py
│ │ │ │ ├── use_schema.py
│ │ │ │ └── wrappers.py
│ │ │ └── logical/
│ │ │ ├── __init__.py
│ │ │ ├── aggregate.py
│ │ │ ├── cross_join.py
│ │ │ ├── empty.py
│ │ │ ├── explain.py
│ │ │ ├── filter.py
│ │ │ ├── join.py
│ │ │ ├── limit.py
│ │ │ ├── project.py
│ │ │ ├── sample.py
│ │ │ ├── sort.py
│ │ │ ├── subquery_alias.py
│ │ │ ├── table_scan.py
│ │ │ ├── union.py
│ │ │ ├── values.py
│ │ │ └── window.py
│ │ ├── rex/
│ │ │ ├── __init__.py
│ │ │ ├── base.py
│ │ │ ├── convert.py
│ │ │ └── core/
│ │ │ ├── __init__.py
│ │ │ ├── alias.py
│ │ │ ├── call.py
│ │ │ ├── input_ref.py
│ │ │ ├── literal.py
│ │ │ └── subquery.py
│ │ └── utils/
│ │ ├── __init__.py
│ │ ├── filter.py
│ │ ├── groupby.py
│ │ ├── ml_classes.py
│ │ ├── sort.py
│ │ └── statistics.py
│ ├── server/
│ │ ├── __init__.py
│ │ ├── app.py
│ │ ├── presto_jdbc.py
│ │ └── responses.py
│ ├── sql-schema.yaml
│ ├── sql.yaml
│ └── utils.py
├── docs/
│ ├── Makefile
│ ├── environment.yml
│ ├── make.bat
│ ├── requirements-docs.txt
│ └── source/
│ ├── api.rst
│ ├── best_practices.rst
│ ├── cmd.rst
│ ├── conf.py
│ ├── configuration.rst
│ ├── custom.rst
│ ├── data_input.rst
│ ├── fugue.rst
│ ├── how_does_it_work.rst
│ ├── index.rst
│ ├── installation.rst
│ ├── machine_learning.rst
│ ├── quickstart.rst
│ ├── server.rst
│ ├── sql/
│ │ ├── creation.rst
│ │ ├── describe.rst
│ │ ├── ml.rst
│ │ └── select.rst
│ └── sql.rst
├── notebooks/
│ ├── Custom Functions.ipynb
│ ├── Feature Overview.ipynb
│ ├── FugueSQL.ipynb
│ └── iris.csv
├── pyproject.toml
├── rustfmt.toml
├── setup.cfg
├── src/
│ ├── dialect.rs
│ ├── error.rs
│ ├── expression.rs
│ ├── lib.rs
│ ├── parser.rs
│ ├── sql/
│ │ ├── column.rs
│ │ ├── exceptions.rs
│ │ ├── function.rs
│ │ ├── logical/
│ │ │ ├── aggregate.rs
│ │ │ ├── alter_schema.rs
│ │ │ ├── alter_table.rs
│ │ │ ├── analyze_table.rs
│ │ │ ├── create_catalog_schema.rs
│ │ │ ├── create_experiment.rs
│ │ │ ├── create_memory_table.rs
│ │ │ ├── create_model.rs
│ │ │ ├── create_table.rs
│ │ │ ├── describe_model.rs
│ │ │ ├── drop_model.rs
│ │ │ ├── drop_schema.rs
│ │ │ ├── drop_table.rs
│ │ │ ├── empty_relation.rs
│ │ │ ├── explain.rs
│ │ │ ├── export_model.rs
│ │ │ ├── filter.rs
│ │ │ ├── join.rs
│ │ │ ├── limit.rs
│ │ │ ├── predict_model.rs
│ │ │ ├── projection.rs
│ │ │ ├── repartition_by.rs
│ │ │ ├── show_columns.rs
│ │ │ ├── show_models.rs
│ │ │ ├── show_schemas.rs
│ │ │ ├── show_tables.rs
│ │ │ ├── sort.rs
│ │ │ ├── subquery_alias.rs
│ │ │ ├── table_scan.rs
│ │ │ ├── use_schema.rs
│ │ │ └── window.rs
│ │ ├── logical.rs
│ │ ├── optimizer/
│ │ │ ├── decorrelate_where_exists.rs
│ │ │ ├── decorrelate_where_in.rs
│ │ │ ├── dynamic_partition_pruning.rs
│ │ │ ├── join_reorder.rs
│ │ │ └── utils.rs
│ │ ├── optimizer.rs
│ │ ├── parser_utils.rs
│ │ ├── preoptimizer.rs
│ │ ├── schema.rs
│ │ ├── statement.rs
│ │ ├── table.rs
│ │ ├── types/
│ │ │ ├── rel_data_type.rs
│ │ │ └── rel_data_type_field.rs
│ │ └── types.rs
│ └── sql.rs
└── tests/
├── __init__.py
├── integration/
│ ├── __init__.py
│ ├── fixtures.py
│ ├── test_analyze.py
│ ├── test_cmd.py
│ ├── test_compatibility.py
│ ├── test_complex.py
│ ├── test_create.py
│ ├── test_distributeby.py
│ ├── test_explain.py
│ ├── test_filter.py
│ ├── test_fugue.py
│ ├── test_function.py
│ ├── test_groupby.py
│ ├── test_hive.py
│ ├── test_intake.py
│ ├── test_jdbc.py
│ ├── test_join.py
│ ├── test_model.py
│ ├── test_over.py
│ ├── test_postgres.py
│ ├── test_rex.py
│ ├── test_sample.py
│ ├── test_schema.py
│ ├── test_select.py
│ ├── test_server.py
│ ├── test_show.py
│ ├── test_sort.py
│ ├── test_sqlite.py
│ └── test_union.py
├── unit/
│ ├── __init__.py
│ ├── test_call.py
│ ├── test_config.py
│ ├── test_context.py
│ ├── test_datacontainer.py
│ ├── test_mapping.py
│ ├── test_ml_utils.py
│ ├── test_queries.py
│ ├── test_statistics.py
│ └── test_utils.py
└── utils.py
================================================
FILE CONTENTS
================================================
================================================
FILE: .cargo/config.toml
================================================
[target.x86_64-apple-darwin]
rustflags = [
"-C", "link-arg=-undefined",
"-C", "link-arg=dynamic_lookup",
]
[target.aarch64-apple-darwin]
rustflags = [
"-C", "link-arg=-undefined",
"-C", "link-arg=dynamic_lookup",
]
================================================
FILE: .coveragerc
================================================
[run]
omit = tests/*
branch = True
[report]
# Regexes for lines to exclude from consideration
exclude_lines =
# Have to re-enable the standard pragma
pragma: no cover
# Don't complain about missing debug-only code:
def __repr__
# Don't complain if tests don't hit defensive assertion code:
raise AssertionError
raise NotImplementedError
# Don't complain if non-runnable code isn't run:
if __name__ == .__main__.:
================================================
FILE: .dockerignore
================================================
node_modules
.next
================================================
FILE: .github/CODEOWNERS
================================================
# global codeowners
* @ayushdg @charlesbluca @galipremsagar
# rust codeowners
.cargo/ @ayushdg @charlesbluca @galipremsagar @jdye64
src/ @ayushdg @charlesbluca @galipremsagar @jdye64
Cargo.toml @ayushdg @charlesbluca @galipremsagar @jdye64
Cargo.lock @ayushdg @charlesbluca @galipremsagar @jdye64
================================================
FILE: .github/ISSUE_TEMPLATE/bug_report.md
================================================
---
name: Bug report
about: Create a bug report to help us improve dask-sql
title: "[BUG]"
labels: "bug, needs triage"
assignees: ''
---
<!-- Please include a self-contained copy-pastable example that generates the issue if possible.
Please be concise with code posted. See guidelines below on how to provide a good bug report:
- Craft Minimal Bug Reports http://matthewrocklin.com/blog/work/2018/02/28/minimal-bug-reports
- Minimal Complete Verifiable Examples https://stackoverflow.com/help/mcve
Bug reports that follow these guidelines are easier to diagnose, and so are often handled much more quickly.
-->
**What happened**:
**What you expected to happen**:
**Minimal Complete Verifiable Example**:
```python
# Put your MCVE code here
```
**Anything else we need to know?**:
**Environment**:
- dask-sql version:
- Python version:
- Operating System:
- Install method (conda, pip, source):
================================================
FILE: .github/ISSUE_TEMPLATE/documentation-request.md
================================================
---
name: Documentation request
about: Report incorrect or needed documentation
title: "[DOC]"
labels: "documentation"
assignees: ''
---
## Report incorrect documentation
**Location of incorrect documentation**
Provide links and line numbers if applicable.
**Describe the problems or issues found in the documentation**
A clear and concise description of what you found to be incorrect.
**Steps taken to verify documentation is incorrect**
List any steps you have taken:
**Suggested fix for documentation**
Detail proposed changes to fix the documentation if you have any.
---
## Report needed documentation
**Report needed documentation**
A clear and concise description of what documentation you believe it is needed and why.
**Describe the documentation you'd like**
A clear and concise description of what you want to happen.
**Steps taken to search for needed documentation**
List any steps you have taken:
================================================
FILE: .github/ISSUE_TEMPLATE/feature_request.md
================================================
---
name: Feature request
about: Suggest an idea for dask-sql
title: "[ENH]"
labels: "enhancement, needs triage"
assignees: ''
---
**Is your feature request related to a problem? Please describe.**
A clear and concise description of what the problem is. Ex. I wish I could use dask-sql to do [...]
**Describe the solution you'd like**
A clear and concise description of what you want to happen.
**Describe alternatives you've considered**
A clear and concise description of any alternative solutions or features you've considered.
**Additional context**
Add any other context, code examples, or references to existing implementations about the feature request here.
================================================
FILE: .github/ISSUE_TEMPLATE/submit-question.md
================================================
---
name: Submit question
about: Ask a general question about dask-sql
title: "[QST]"
labels: "question"
assignees: ''
---
**What is your question?**
================================================
FILE: .github/dependabot.yml
================================================
version: 2
updates:
- package-ecosystem: "cargo"
directory: "/"
schedule:
interval: "daily"
ignore:
# arrow and datafusion are bumped manually
- dependency-name: "arrow"
update-types: ["version-update:semver-major"]
- dependency-name: "datafusion"
update-types: ["version-update:semver-major"]
- dependency-name: "datafusion-*"
update-types: ["version-update:semver-major"]
- package-ecosystem: "github-actions"
directory: "/"
schedule:
# Check for updates to GitHub Actions every weekday
interval: "weekly"
ignore:
# prefer updating cibuildwheel manually as needed
- dependency-name: "pypa/cibuildwheel"
================================================
FILE: .github/workflows/conda.yml
================================================
name: Build conda nightly
on:
push:
branches:
- main
pull_request:
paths:
- Cargo.toml
- Cargo.lock
- pyproject.toml
- continuous_integration/recipe/**
- .github/workflows/conda.yml
schedule:
- cron: '0 0 * * 0'
# When this workflow is queued, automatically cancel any previous running
# or pending jobs from the same branch
concurrency:
group: conda-${{ github.head_ref }}
cancel-in-progress: true
# Required shell entrypoint to have properly activated conda environments
defaults:
run:
shell: bash -l {0}
jobs:
conda:
name: "Build conda nightlies (python: ${{ matrix.python }}, arch: ${{ matrix.arch }})"
runs-on: ubuntu-latest
strategy:
fail-fast: false
matrix:
python: ["3.9", "3.10", "3.11", "3.12"]
arch: ["linux-64", "linux-aarch64"]
steps:
- name: Manage disk space
if: matrix.arch == 'linux-aarch64'
run: |
sudo mkdir -p /opt/empty_dir || true
for d in \
/opt/ghc \
/opt/hostedtoolcache \
/usr/lib/jvm \
/usr/local/.ghcup \
/usr/local/lib/android \
/usr/local/share/powershell \
/usr/share/dotnet \
/usr/share/swift \
; do
sudo rsync --stats -a --delete /opt/empty_dir/ $d || true
done
sudo apt-get purge -y -f firefox \
google-chrome-stable \
microsoft-edge-stable
sudo apt-get autoremove -y >& /dev/null
sudo apt-get autoclean -y >& /dev/null
sudo docker image prune --all --force
df -h
- name: Create swapfile
if: matrix.arch == 'linux-aarch64'
run: |
sudo fallocate -l 10GiB /swapfile || true
sudo chmod 600 /swapfile || true
sudo mkswap /swapfile || true
sudo swapon /swapfile || true
- uses: actions/checkout@v4
with:
fetch-depth: 0
- name: Set up Python
uses: conda-incubator/setup-miniconda@v2.3.0
with:
miniforge-variant: Mambaforge
use-mamba: true
python-version: "3.9"
channel-priority: strict
- name: Install dependencies
run: |
mamba install -c conda-forge "boa<0.17" "conda-build<24.1" conda-verify
which python
pip list
mamba list
- name: Build conda packages
run: |
# suffix for nightly package versions
export VERSION_SUFFIX=a`date +%y%m%d`
conda mambabuild continuous_integration/recipe \
--python ${{ matrix.python }} \
--variants "{target_platform: [${{ matrix.arch }}]}" \
--error-overlinking \
--no-test \
--no-anaconda-upload \
--output-folder packages
- name: Test conda packages
if: matrix.arch == 'linux-64' # can only test native platform packages
run: |
conda mambabuild --test packages/${{ matrix.arch }}/*.tar.bz2
- name: Upload conda packages as artifacts
uses: actions/upload-artifact@v3
with:
name: "conda nightlies (python - ${{ matrix.python }}, arch - ${{ matrix.arch }})"
# need to install all conda channel metadata to properly install locally
path: packages/
- name: Upload conda packages to Anaconda
if: |
github.event_name == 'push'
&& github.repository == 'dask-contrib/dask-sql'
env:
ANACONDA_API_TOKEN: ${{ secrets.DASK_CONDA_TOKEN }}
run: |
# install anaconda for upload
mamba install -c conda-forge anaconda-client
anaconda upload --label dev packages/${{ matrix.arch }}/*.tar.bz2
================================================
FILE: .github/workflows/docker.yml
================================================
name: Build Docker image
on:
release:
types: [created]
push:
branches:
- main
pull_request:
paths:
- Cargo.toml
- Cargo.lock
- pyproject.toml
- continuous_integration/docker/**
- .github/workflows/docker.yml
# When this workflow is queued, automatically cancel any previous running
# or pending jobs from the same branch
concurrency:
group: docker-${{ github.ref }}
cancel-in-progress: true
jobs:
push_to_registry:
name: Push Docker image to Docker Hub
runs-on: ubuntu-latest
env:
DOCKER_PUSH: ${{ contains(fromJSON('["push", "release"]'), github.event_name) && github.repository == 'dask-contrib/dask-sql' }}
strategy:
fail-fast: false
matrix:
platform: ["linux/amd64", "linux/arm64", "linux/386"]
steps:
- uses: actions/checkout@v4
- name: Login to DockerHub
if: ${{ fromJSON(env.DOCKER_PUSH) }}
uses: docker/login-action@v3
with:
username: ${{ secrets.DOCKER_USERNAME }}
password: ${{ secrets.DOCKER_PASSWORD }}
- name: Docker meta for main image
id: docker_meta_main
uses: crazy-max/ghaction-docker-meta@v5
with:
images: nbraun/dask-sql
- name: Build and push main image
uses: docker/build-push-action@v5
with:
context: .
file: ./continuous_integration/docker/main.dockerfile
build-args: DOCKER_META_VERSION=${{ steps.docker_meta_main.outputs.version }}
platforms: ${{ matrix.platform }}
tags: ${{ steps.docker_meta_main.outputs.tags }}
labels: ${{ steps.docker_meta_main.outputs.labels }}
push: ${{ fromJSON(env.DOCKER_PUSH) }}
load: ${{ !fromJSON(env.DOCKER_PUSH) }}
- name: Check images
run: |
df -h
docker image ls
docker image inspect ${{ steps.docker_meta_main.outputs.tags }}
- name: Docker meta for cloud image
id: docker_meta_cloud
uses: crazy-max/ghaction-docker-meta@v5
with:
images: nbraun/dask-sql-cloud
- name: Build and push cloud image
uses: docker/build-push-action@v5
with:
context: .
file: ./continuous_integration/docker/cloud.dockerfile
build-args: DOCKER_META_VERSION=${{ steps.docker_meta_main.outputs.version }}
platforms: ${{ matrix.platform }}
tags: ${{ steps.docker_meta_cloud.outputs.tags }}
labels: ${{ steps.docker_meta_cloud.outputs.labels }}
push: ${{ fromJSON(env.DOCKER_PUSH) }}
load: ${{ !fromJSON(env.DOCKER_PUSH) }}
================================================
FILE: .github/workflows/release.yml
================================================
name: Upload Python package
on:
release:
types: [created]
pull_request:
paths:
- .github/workflows/release.yml
- dask_sql/__init__.py
# When this workflow is queued, automatically cancel any previous running
# or pending jobs from the same branch
concurrency:
group: release-${{ github.head_ref }}
cancel-in-progress: true
env:
upload: ${{ github.event_name == 'release' && github.repository == 'dask-contrib/dask-sql' }}
jobs:
linux:
name: Build and publish wheels for linux ${{ matrix.target }}
runs-on: ubuntu-latest
strategy:
fail-fast: false
matrix:
target: [x86_64, aarch64]
steps:
- uses: actions/checkout@v4
- uses: actions/setup-python@v4
with:
python-version: '3.10'
- name: Build wheels for x86_64
if: matrix.target == 'x86_64'
uses: PyO3/maturin-action@v1
with:
target: ${{ matrix.target }}
args: --release --out dist
sccache: 'true'
manylinux: '2_17'
- name: Build wheels for aarch64
if: matrix.target == 'aarch64'
uses: PyO3/maturin-action@v1
with:
target: ${{ matrix.target }}
args: --release --out dist --zig
sccache: 'true'
manylinux: '2_17'
- name: Check dist files
run: |
pip install twine
twine check dist/*
ls -lh dist/
- name: Upload binary wheels
uses: actions/upload-artifact@v3
with:
name: wheels for linux ${{ matrix.target }}
path: dist/*
- name: Publish package
if: env.upload == 'true'
env:
TWINE_USERNAME: ${{ secrets.PYPI_USERNAME }}
TWINE_PASSWORD: ${{ secrets.PYPI_PASSWORD }}
run: twine upload dist/*
windows:
name: Build and publish wheels for windows
runs-on: windows-latest
steps:
- uses: actions/checkout@v4
- uses: actions/setup-python@v4
with:
python-version: '3.10'
architecture: x64
- name: Build wheels
uses: PyO3/maturin-action@v1
with:
target: x64
args: --release --out dist
sccache: 'true'
- name: Check dist files
run: |
pip install twine
twine check dist/*
ls dist/
- name: Upload binary wheels
uses: actions/upload-artifact@v3
with:
name: wheels for windows
path: dist/*
- name: Publish package
if: env.upload == 'true'
env:
TWINE_USERNAME: ${{ secrets.PYPI_USERNAME }}
TWINE_PASSWORD: ${{ secrets.PYPI_PASSWORD }}
run: twine upload dist/*
macos:
name: Build and publish wheels for macos ${{ matrix.target }}
runs-on: macos-latest
strategy:
fail-fast: false
matrix:
target: [x86_64, aarch64]
steps:
- uses: actions/checkout@v4
- uses: actions/setup-python@v4
with:
python-version: '3.10'
- name: Build wheels
uses: PyO3/maturin-action@v1
with:
target: ${{ matrix.target }}
args: --release --out dist
sccache: 'true'
- name: Check dist files
run: |
pip install twine
twine check dist/*
ls -lh dist/
- name: Upload binary wheels
uses: actions/upload-artifact@v3
with:
name: wheels for macos ${{ matrix.target }}
path: dist/*
- name: Publish package
if: env.upload == 'true'
env:
TWINE_USERNAME: ${{ secrets.PYPI_USERNAME }}
TWINE_PASSWORD: ${{ secrets.PYPI_PASSWORD }}
run: twine upload dist/*
sdist:
runs-on: ubuntu-latest
steps:
- uses: actions/checkout@v4
- name: Build sdist
uses: PyO3/maturin-action@v1
with:
command: sdist
args: --out dist
- uses: actions/setup-python@v4
with:
python-version: '3.10'
- name: Check dist files
run: |
pip install twine
twine check dist/*
ls -lh dist/
- name: Publish source distribution
if: env.upload == 'true'
env:
TWINE_USERNAME: ${{ secrets.PYPI_USERNAME }}
TWINE_PASSWORD: ${{ secrets.PYPI_PASSWORD }}
run: twine upload dist/*
================================================
FILE: .github/workflows/rust.yml
================================================
name: Test Rust package
on:
# always trigger on PR
push:
branches:
- main
pull_request:
# manual trigger
# https://docs.github.com/en/actions/managing-workflow-runs/manually-running-a-workflow
workflow_dispatch:
env:
# Disable full debug symbol generation to speed up CI build and keep memory down
# "1" means line tables only, which is useful for panic tracebacks.
RUSTFLAGS: "-C debuginfo=1"
jobs:
detect-ci-trigger:
name: Check for upstream trigger phrase
runs-on: ubuntu-latest
if: github.repository == 'dask-contrib/dask-sql'
outputs:
triggered: ${{ steps.detect-trigger.outputs.trigger-found }}
steps:
- uses: actions/checkout@v4
with:
fetch-depth: 2
- uses: xarray-contrib/ci-trigger@v1.2
id: detect-trigger
with:
keyword: "[test-df-upstream]"
# Check crate compiles
linux-build-lib:
name: cargo check
needs: [detect-ci-trigger]
runs-on: ubuntu-latest
steps:
- uses: actions/checkout@v4
- uses: actions-rs/toolchain@v1
with:
toolchain: 1.72
default: true
- name: Cache Cargo
uses: actions/cache@v3
with:
path: /home/runner/.cargo
key: cargo-cache
- name: Optionally update upstream dependencies
if: needs.detect-ci-trigger.outputs.triggered == 'true'
run: |
bash continuous_integration/scripts/update-dependencies.sh
- name: Check workspace in debug mode
run: |
cargo check
- name: Check workspace in release mode
run: |
cargo check --release
# test the crate
linux-test:
name: cargo test
needs: [detect-ci-trigger]
runs-on: ubuntu-latest
steps:
- uses: actions/checkout@v4
with:
submodules: true
- uses: actions-rs/toolchain@v1
with:
toolchain: 1.72
default: true
- name: Cache Cargo
uses: actions/cache@v3
with:
path: /home/runner/.cargo
key: cargo-cache
- name: Optionally update upstream dependencies
if: needs.detect-ci-trigger.outputs.triggered == 'true'
run: |
bash continuous_integration/scripts/update-dependencies.sh
- name: Run tests
run: |
cargo test
================================================
FILE: .github/workflows/style.yml
================================================
---
name: Python style check
on: [pull_request]
# When this workflow is queued, automatically cancel any previous running
# or pending jobs from the same branch
concurrency:
group: style-${{ github.head_ref }}
cancel-in-progress: true
jobs:
pre-commit:
name: Run pre-commit hooks
runs-on: ubuntu-latest
steps:
- uses: actions/checkout@v4
- uses: actions/setup-python@v4
- uses: actions-rs/toolchain@v1
with:
toolchain: 1.72
components: clippy
default: true
- uses: actions-rs/toolchain@v1
with:
toolchain: nightly
components: rustfmt
- uses: pre-commit/action@v3.0.0
================================================
FILE: .github/workflows/test-upstream.yml
================================================
name: Nightly upstream testing
on:
schedule:
- cron: "0 0 * * *" # Daily “At 00:00” UTC
workflow_dispatch: # allows you to trigger the workflow run manually
# Required shell entrypoint to have properly activated conda environments
defaults:
run:
shell: bash -l {0}
jobs:
test-dev:
name: "Test upstream dev (${{ matrix.os }}, python: ${{ matrix.python }}, distributed: ${{ matrix.distributed }}, query-planning: ${{ matrix.query-planning }})"
runs-on: ${{ matrix.os }}
env:
CONDA_FILE: continuous_integration/environment-${{ matrix.python }}.yaml
DASK_SQL_DISTRIBUTED_TESTS: ${{ matrix.distributed }}
DASK_DATAFRAME__QUERY_PLANNING: ${{ matrix.query-planning }}
strategy:
fail-fast: false
matrix:
os: [ubuntu-latest, windows-latest, macos-latest]
python: ["3.9", "3.10", "3.11", "3.12"]
distributed: [false]
query-planning: [true]
include:
# run tests on a distributed client
- os: "ubuntu-latest"
python: "3.9"
distributed: true
query-planning: true
- os: "ubuntu-latest"
python: "3.11"
distributed: true
query-planning: true
# run tests with query planning disabled
- os: "ubuntu-latest"
python: "3.9"
distributed: false
query-planning: false
- os: "ubuntu-latest"
python: "3.11"
distributed: false
query-planning: false
steps:
- uses: actions/checkout@v4
with:
fetch-depth: 0 # Fetch all history for all branches and tags.
- name: Set up Python
uses: conda-incubator/setup-miniconda@v2.3.0
with:
miniforge-variant: Mambaforge
use-mamba: true
python-version: ${{ matrix.python }}
channel-priority: strict
activate-environment: dask-sql
environment-file: ${{ env.CONDA_FILE }}
- uses: actions-rs/toolchain@v1
with:
toolchain: 1.72
default: true
- name: Install x86_64-apple-darwin target
if: matrix.os == 'macos-latest'
run: rustup target add x86_64-apple-darwin
- name: Build the Rust DataFusion bindings
run: |
maturin develop
- name: Install hive testing dependencies
if: matrix.os == 'ubuntu-latest'
run: |
docker pull bde2020/hive:2.3.2-postgresql-metastore
docker pull bde2020/hive-metastore-postgresql:2.3.0
- name: Install upstream dev Dask
run: |
mamba install --no-channel-priority dask/label/dev::dask
- name: Install pytest-reportlog
run: |
# TODO: add pytest-reportlog to testing environments if we move over to JSONL output
mamba install pytest-reportlog
- name: Test with pytest
id: run_tests
run: |
pytest --report-log test-${{ matrix.os }}-py${{ matrix.python }}-results.jsonl --cov-report=xml -n auto tests --dist loadfile
- name: Upload pytest results for failure
if: |
always()
&& steps.run_tests.outcome != 'skipped'
uses: actions/upload-artifact@v3
with:
name: test-${{ matrix.os }}-py${{ matrix.python }}-results
path: test-${{ matrix.os }}-py${{ matrix.python }}-results.jsonl
import-dev:
name: "Test importing with bare requirements and upstream dev (query-planning: ${{ matrix.query-planning }})"
runs-on: ubuntu-latest
strategy:
fail-fast: false
matrix:
query-planning: [true, false]
steps:
- uses: actions/checkout@v4
- name: Set up Python
uses: conda-incubator/setup-miniconda@v2.3.0
with:
miniforge-variant: Mambaforge
use-mamba: true
python-version: "3.9"
channel-priority: strict
- uses: actions-rs/toolchain@v1
with:
toolchain: 1.72
default: true
- name: Install dependencies and nothing else
run: |
pip install -e . -vv
which python
pip list
mamba list
- name: Install upstream dev Dask
run: |
python -m pip install git+https://github.com/dask/dask
python -m pip install git+https://github.com/dask/dask-expr
python -m pip install git+https://github.com/dask/distributed
- name: Try to import dask-sql
env:
DASK_DATAFRAME_QUERY_PLANNING: ${{ matrix.query-planning }}
run: |
python -c "import dask_sql; print('ok')"
report-failures:
name: Open issue for upstream dev failures
needs: [test-dev, import-dev]
if: |
always()
&& (
needs.test-dev.result == 'failure'
|| needs.import-dev.result == 'failure'
)
&& github.repository == 'dask-contrib/dask-sql'
runs-on: ubuntu-latest
steps:
- uses: actions/checkout@v4
- uses: actions/download-artifact@v3
- name: Prepare logs & issue label
run: |
# TODO: remove this if xarray-contrib/issue-from-pytest-log no longer needs a log-path
if [ -f test-ubuntu-latest-py3.10-results/test-ubuntu-latest-py3.10-results.jsonl ]; then
cp test-ubuntu-latest-py3.10-results/test-ubuntu-latest-py3.10-results.jsonl results.jsonl
else
touch results.jsonl
fi
- name: Open or update issue on failure
uses: xarray-contrib/issue-from-pytest-log@v1.2.6
with:
log-path: results.jsonl
issue-title: ⚠️ Upstream CI failed ⚠️
issue-label: upstream
================================================
FILE: .github/workflows/test.yml
================================================
name: Test Python package
on:
push:
branches:
- main
pull_request:
# When this workflow is queued, automatically cancel any previous running
# or pending jobs from the same branch
concurrency:
group: test-${{ github.head_ref }}
cancel-in-progress: true
# Required shell entrypoint to have properly activated conda environments
defaults:
run:
shell: bash -l {0}
jobs:
detect-ci-trigger:
name: Check for upstream trigger phrase
runs-on: ubuntu-latest
if: github.repository == 'dask-contrib/dask-sql'
outputs:
triggered: ${{ steps.detect-trigger.outputs.trigger-found }}
steps:
- uses: actions/checkout@v4
with:
fetch-depth: 2
- uses: xarray-contrib/ci-trigger@v1.2
id: detect-trigger
with:
keyword: "[test-upstream]"
test:
name: "Build & Test (${{ matrix.os }}, python: ${{ matrix.python }}, distributed: ${{ matrix.distributed }}, query-planning: ${{ matrix.query-planning }})"
needs: [detect-ci-trigger]
runs-on: ${{ matrix.os }}
env:
CONDA_FILE: continuous_integration/environment-${{ matrix.python }}.yaml
DASK_SQL_DISTRIBUTED_TESTS: ${{ matrix.distributed }}
DASK_DATAFRAME__QUERY_PLANNING: ${{ matrix.query-planning }}
strategy:
fail-fast: false
matrix:
os: [ubuntu-latest, windows-latest, macos-latest]
python: ["3.9", "3.10", "3.11", "3.12"]
distributed: [false]
query-planning: [true]
include:
# run tests on a distributed client
- os: "ubuntu-latest"
python: "3.9"
distributed: true
query-planning: true
- os: "ubuntu-latest"
python: "3.11"
distributed: true
query-planning: true
# run tests with query planning disabled
- os: "ubuntu-latest"
python: "3.9"
distributed: false
query-planning: false
- os: "ubuntu-latest"
python: "3.11"
distributed: false
query-planning: false
steps:
- uses: actions/checkout@v4
- name: Set up Python
uses: conda-incubator/setup-miniconda@v2.3.0
with:
miniforge-variant: Mambaforge
use-mamba: true
python-version: ${{ matrix.python }}
channel-priority: strict
activate-environment: dask-sql
environment-file: ${{ env.CONDA_FILE }}
run-post: ${{ matrix.os != 'windows-latest' && 'true' || 'false' }}
- uses: actions-rs/toolchain@v1
with:
toolchain: 1.72
default: true
- name: Install x86_64-apple-darwin target
if: matrix.os == 'macos-latest'
run: rustup target add x86_64-apple-darwin
- name: Build the Rust DataFusion bindings
run: |
maturin develop
- name: Install hive testing dependencies
if: matrix.os == 'ubuntu-latest'
run: |
docker pull bde2020/hive:2.3.2-postgresql-metastore
docker pull bde2020/hive-metastore-postgresql:2.3.0
- name: Optionally install upstream dev Dask
if: needs.detect-ci-trigger.outputs.triggered == 'true'
run: |
mamba install --no-channel-priority dask/label/dev::dask
- name: Test with pytest
run: |
pytest --junitxml=junit/test-results.xml --cov-report=xml -n auto tests --dist loadfile
- name: Upload pytest test results
if: always()
uses: actions/upload-artifact@v3
with:
name: pytest-results
path: junit/test-results.xml
- name: Upload coverage to Codecov
if: github.repository == 'dask-contrib/dask-sql'
uses: codecov/codecov-action@v3
import:
name: "Test importing with bare requirements (query-planning: ${{ matrix.query-planning }})"
needs: [detect-ci-trigger]
runs-on: ubuntu-latest
strategy:
fail-fast: false
matrix:
query-planning: [true, false]
steps:
- uses: actions/checkout@v4
- name: Set up Python
uses: conda-incubator/setup-miniconda@v2.3.0
with:
miniforge-variant: Mambaforge
use-mamba: true
python-version: "3.9"
channel-priority: strict
- uses: actions-rs/toolchain@v1
with:
toolchain: 1.72
default: true
- name: Install dependencies and nothing else
run: |
pip install -e . -vv
which python
pip list
mamba list
- name: Optionally install upstream dev Dask
if: needs.detect-ci-trigger.outputs.triggered == 'true'
run: |
python -m pip install git+https://github.com/dask/dask
python -m pip install git+https://github.com/dask/dask-expr
python -m pip install git+https://github.com/dask/distributed
- name: Try to import dask-sql
env:
DASK_DATAFRAME_QUERY_PLANNING: ${{ matrix.query-planning }}
run: |
python -c "import dask_sql; print('ok')"
================================================
FILE: .gitignore
================================================
# Byte-compiled / optimized / DLL files
__pycache__/
*.py[cod]
*$py.class
# Distribution / packaging
.Python
build/
develop-eggs/
dist/
downloads/
eggs/
.eggs/
lib/
lib64/
parts/
sdist/
var/
wheels/
share/python-wheels/
*.egg-info/
.installed.cfg
*.egg
MANIFEST
*.so
# Unit test / coverage reports
htmlcov/
.coverage
.coverage.*
.cache
coverage.xml
*.cover
.pytest_cache/
.hypothesis/
.pytest-html
# Jupyter Notebook
.ipynb_checkpoints
# environments
conda-env
env
venv
# IDE
.idea
.vscode
*.swp
# project specific
dask-worker-space/
node_modules/
docs/source/_build/
tests/unit/queries
tests/unit/data
target/*
packages/*
# Ignore development specific local testing files
dev_tests
dev-tests
================================================
FILE: .pre-commit-config.yaml
================================================
repos:
- repo: https://github.com/psf/black
rev: 22.10.0
hooks:
- id: black
language_version: python3
- repo: https://github.com/PyCQA/flake8
rev: 5.0.4
hooks:
- id: flake8
language_version: python3
- repo: https://github.com/pycqa/isort
rev: 5.12.0
hooks:
- id: isort
args:
- "--profile"
- "black"
- repo: https://github.com/doublify/pre-commit-rust
rev: v1.0
hooks:
- id: cargo-check
args: ['--manifest-path', './Cargo.toml', '--verbose', '--']
- id: clippy
args: ['--manifest-path', './Cargo.toml', '--verbose', '--', '-D', 'warnings']
- repo: https://github.com/pre-commit/pre-commit-hooks
rev: v4.2.0
hooks:
- id: trailing-whitespace
- id: end-of-file-fixer
- id: check-yaml
exclude: ^continuous_integration/recipe/
- id: check-added-large-files
- repo: local
hooks:
- id: cargo-fmt
name: cargo fmt
description: Format files with cargo fmt.
entry: cargo +nightly fmt
language: system
types: [rust]
args: ['--manifest-path', './Cargo.toml', '--verbose', '--']
================================================
FILE: .readthedocs.yaml
================================================
# See https://docs.readthedocs.io/en/stable/config-file/v2.html for details
version: 2
build:
os: ubuntu-20.04
tools:
python: "mambaforge-4.10"
sphinx:
configuration: docs/source/conf.py
conda:
environment: docs/environment.yml
python:
install:
- method: pip
path: .
================================================
FILE: CODE_OF_CONDUCT.md
================================================
# Contributor Covenant Code of Conduct
## Our Pledge
In the interest of fostering an open and welcoming environment, we as
contributors and maintainers pledge to making participation in our project and
our community a harassment-free experience for everyone, regardless of age, body
size, disability, ethnicity, sex characteristics, gender identity and expression,
level of experience, education, socio-economic status, nationality, personal
appearance, race, religion, or sexual identity and orientation.
## Our Standards
Examples of behavior that contributes to creating a positive environment
include:
* Using welcoming and inclusive language
* Being respectful of differing viewpoints and experiences
* Gracefully accepting constructive criticism
* Focusing on what is best for the community
* Showing empathy towards other community members
Examples of unacceptable behavior by participants include:
* The use of sexualized language or imagery and unwelcome sexual attention or
advances
* Trolling, insulting/derogatory comments, and personal or political attacks
* Public or private harassment
* Publishing others' private information, such as a physical or electronic
address, without explicit permission
* Other conduct which could reasonably be considered inappropriate in a
professional setting
## Our Responsibilities
Project maintainers are responsible for clarifying the standards of acceptable
behavior and are expected to take appropriate and fair corrective action in
response to any instances of unacceptable behavior.
Project maintainers have the right and responsibility to remove, edit, or
reject comments, commits, code, wiki edits, issues, and other contributions
that are not aligned to this Code of Conduct, or to ban temporarily or
permanently any contributor for other behaviors that they deem inappropriate,
threatening, offensive, or harmful.
## Scope
This Code of Conduct applies both within project spaces and in public spaces
when an individual is representing the project or its community. Examples of
representing a project or community include using an official project e-mail
address, posting via an official social media account, or acting as an appointed
representative at an online or offline event. Representation of a project may be
further defined and clarified by project maintainers.
## Enforcement
Instances of abusive, harassing, or otherwise unacceptable behavior may be
reported by contacting the project team at nilslennartbraun@gmail.com. All
complaints will be reviewed and investigated and will result in a response that
is deemed necessary and appropriate to the circumstances. The project team is
obligated to maintain confidentiality with regard to the reporter of an incident.
Further details of specific enforcement policies may be posted separately.
Project maintainers who do not follow or enforce the Code of Conduct in good
faith may face temporary or permanent repercussions as determined by other
members of the project's leadership.
## Attribution
This Code of Conduct is adapted from the [Contributor Covenant][homepage], version 1.4,
available at https://www.contributor-covenant.org/version/1/4/code-of-conduct.html
[homepage]: https://www.contributor-covenant.org
For answers to common questions about this code of conduct, see
https://www.contributor-covenant.org/faq
================================================
FILE: CONTRIBUTING.md
================================================
# Contributing to Dask-SQL
## Environment Setup
The environment used for development and CI consists of:
- a system installation of [`rustup`](https://rustup.rs/) with:
- the latest stable toolchain
- the latest nightly `rustfmt`
- a [conda](https://docs.conda.io/projects/conda/en/latest/user-guide/install/index.html) environment containing all required Python packages
Once `rustup` is installed, ensure that the latest stable toolchain and nightly `rustfmt` are available by running
```
rustup toolchain install nightly -c rustfmt --profile minimal
rustup update
```
To initialize and activate the conda environment for a given Python version:
```
conda env create -f dask-sql/continuous_integration/environment-{$PYTHON_VER}.yaml
conda activate dask-sql
```
## Rust Developers Guide
Dask-SQL utilizes [Apache Arrow Datafusion](https://github.com/apache/arrow-datafusion) for parsing, planning, and optimizing SQL queries. DataFusion is written in Rust and therefore requires some Rust experience to be productive. Luckily, there are tons of great Rust learning resources on the internet. We have listed some of our favorite ones [here](#rust-learning-resources)
### Apache Arrow DataFusion
The Dask-SQL Rust codebase makes heavy use [Apache Arrow DataFusion](https://github.com/apache/arrow-datafusion). Contributors should familiarize themselves with the [codebase](https://github.com/apache/arrow-datafusion) and [documentation](https://docs.rs/datafusion/latest/datafusion/).
#### Purpose
DataFusion provides Dask-SQL with key functionality.
- Parsing SQL query strings into a `LogicalPlan` datastructure
- Future integration points with [substrait.io](https://substrait.io/)
- An optimization framework used as the baseline for creating custom highly efficient `LogicalPlan`s specific to Dask.
### Building
Building the Dask-SQL Rust codebase is a straightforward process. If you create and activate the Dask-SQL Conda environment the Rust compiler and all necessary components will be installed for you during that process and therefore requires no further manual setup.
`maturin` is used by Dask-SQL for building and bundling the resulting Rust binaries. This helps make building and installing the Rust binaries feel much more like a native Python workflow.
More details about the building setup can be found in [pyproject.toml](pyproject.toml) and [Cargo.toml](Cargo.toml)
Note that while `maturin` is used by CI and should be used during your development cycle, if the need arises to do something more specific that is not yet supported by `maturin` you can opt to use `cargo` directly from the command line.
#### Building with Python
Building Dask-SQL is straightforward with Python. To build run ```pip install .```. This will build both the Rust and Python codebase and install it into your locally activated conda environment; note that if your Rust dependencies have been updated, this command must be rerun to rebuild the Rust codebase.
#### DataFusion Modules
DataFusion is broken down into a few modules. We consume those modules in our [Cargo.toml](Cargo.toml). The modules that we use currently are
- `datafusion-common` - Datastructures and core logic
- `datafusion-expr` - Expression based logic and operators
- `datafusion-sql` - SQL components such as parsing and planning
- `datafusion-optimizer` - Optimization logic and datastructures for modifying current plans into more efficient ones.
#### Retrieving Upstream Dependencies
During development you might find yourself needing some upstream DataFusion changes not present in the projects current version. Luckily this can easily be achieved by updating [Cargo.toml](Cargo.toml) and changing the `rev` to the SHA of the version you need. Note that the same SHA should be used for all DataFusion modules.
#### Local Documentation
Sometimes when building against the latest Github commits for DataFusion you may find that the features you are consuming do not have their documentation public yet. In this case it can be helpful to build the DataFusion documentation locally so that it can be referenced to assist with development. Here is a rough outline for building that documentation locally.
- clone https://github.com/apache/arrow-datafusion
- change into the `arrow-datafusion` directory
- run `cargo doc`
- navigate to `target/doc/datafusion/all.html` and open in your desired browser
### Datastructures
While working in the Rust codebase there are a few datastructures that you should make yourself familiar with. This section does not aim to verbosely list out all of the datastructure with in the project but rather just the key datastructures that you are likely to encounter while working on almost any feature/issue. The aim is to give you a better overview of the codebase without having to manually dig through the all the source code.
- [`PyLogicalPlan`](src/sql/logical.rs) -> [DataFusion LogicalPlan](https://docs.rs/datafusion/latest/datafusion/logical_plan/enum.LogicalPlan.html)
- Often encountered in Python code with variable name `rel`
- Python serializable umbrella representation of the entire LogicalPlan that was generated by DataFusion
- Provides access to `DaskTable` instances and type information for each table
- Access to individual nodes in the logical plan tree. Ex: `TableScan`
- [`DaskSQLContext`](src/sql.rs)
- Analogous to Python `Context`
- Contains metadata about the tables, schemas, functions, operators, and configurations that are persent within the current execution context
- When adding custom functions/UDFs this is the location that you would register them
- Entry point for parsing SQL strings to sql node trees. This is the location Python will begin its interactions with Rust
- [`PyExpr`](src/expression.rs) -> [DataFusion Expr](https://docs.rs/datafusion/latest/datafusion/prelude/enum.Expr.html)
- Arguably where most of your time will be spent
- Represents a single node in sql tree. Ex: `avg(age)` from `SELECT avg(age) FROM people`
- Is associate with a single `RexType`
- Can contain literal values or represent function calls, `avg()` for example
- The expressions "index" in the tree can be retrieved by calling `PyExpr.index()` on an instance. This is useful when mapping frontend column names in Dask code to backend Dataframe columns
- Certain `PyExpr`s contain operands. Ex: `2 + 2` would contain 3 operands. 1) A literal `PyExpr` instance with value 2 2) Another literal `PyExpr` instance with a value of 2. 3) A `+` `PyExpr` representing the addition of the 2 literals.
- [`DaskSqlOptimizer`](src/sql/optimizer.rs)
- Registering location for all Dask-SQL specific logical plan optimizations
- Optimizations that are written either custom or use from another source, DataFusion, are registered here in the order they are wished to be executed
- Represents functions that modify/convert an original `PyLogicalPlan` into another `PyLogicalPlan` that would be more efficient when running in the underlying Dask framework
- [`RelDataType`](src/sql/types/rel_data_type.rs)
- Not a fan of this name, was chosen to match existing Calcite logic
- Represents a "row" in a table
- Contains a list of "columns" that are present in that row
- [RelDataTypeField](src/sql/types/rel_data_type_field.rs)
- [RelDataTypeField](src/sql/types/rel_data_type_field.rs)
- Represents an individual column in a table
- Contains:
- `qualifier` - schema the field belongs to
- `name` - name of the column/field
- `data_type` - `DaskTypeMap` instance containing information about the SQL type and underlying Arrow DataType
- `index` - location of the field in the LogicalPlan
- [DaskTypeMap](src/sql/types.rs)
- Maps a conventional SQL type to an underlying Arrow DataType
### Rust Learning Resources
- ["The Book"](https://doc.rust-lang.org/book/)
- [Lets Get Rusty "LGR" YouTube series](https://www.youtube.com/c/LetsGetRusty)
## Documentation TODO
- [ ] SQL Parsing overview diagram
- [ ] Architecture diagram
- [x] Setup dev environment
- [x] Version of Rust and specs
- [x] Updating version of datafusion
- [x] Building
- [x] Rust learning resources
- [x] Rust Datastructures local to Dask-SQL
- [x] Build DataFusion documentation locally
- [ ] Python & Rust with PyO3
- [ ] Types mapping, Arrow datatypes
- [ ] RexTypes explaination, show simple query and show it broken down into its parts in a diagram
- [ ] Registering tables with DaskSqlContext, also functions
- [ ] Creating your own optimizer
- [ ] Simple diagram of PyExpr, showing something like 2+2 but broken down into a tree looking diagram
================================================
FILE: Cargo.toml
================================================
[package]
name = "dask-sql"
repository = "https://github.com/dask-contrib/dask-sql"
version = "2024.5.0"
description = "Bindings for DataFusion used by Dask-SQL"
readme = "README.md"
license = "Apache-2.0"
edition = "2021"
rust-version = "1.72"
include = ["/src", "/dask_sql", "/LICENSE.txt", "pyproject.toml", "Cargo.toml", "Cargo.lock"]
[dependencies]
async-trait = "0.1.78"
datafusion-python = { git = "https://github.com/apache/arrow-datafusion-python.git", ref = "da6c183" }
env_logger = "0.11"
log = "^0.4"
pyo3 = { version = "0.19.2", features = ["extension-module", "abi3", "abi3-py39"] }
pyo3-log = "0.9.0"
[build-dependencies]
pyo3-build-config = "0.20.3"
[lib]
name = "dask_sql"
crate-type = ["cdylib", "rlib"]
[profile.release]
lto = true
codegen-units = 1
================================================
FILE: LICENSE.txt
================================================
MIT LICENCE
Copyright (c) 2020 Nils Braun
Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit
persons to whom the Software is furnished to do so, subject to the following conditions:
The above copyright notice and this permission notice shall be included in all copies or substantial portions of the
Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE
WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
================================================
FILE: MANIFEST.in
================================================
recursive-include dask_sql *.yaml
recursive-include dask_planner *
================================================
FILE: README.md
================================================
**Dask-SQL is currently not in active maintenance, see [#1344](https://github.com/dask-contrib/dask-sql/issues/1344) for more information**
[](https://anaconda.org/conda-forge/dask-sql)
[](https://pypi.python.org/pypi/dask-sql/)
[](https://github.com/dask-contrib/dask-sql/actions/workflows/test.yml?query=branch%3Amain)
[](https://dask-sql.readthedocs.io/en/latest/)
[](https://codecov.io/gh/dask-contrib/dask-sql)
[](https://github.com/dask-contrib/dask-sql/blob/main/LICENSE.txt)
[](https://mybinder.org/v2/gh/dask-contrib/dask-sql-binder/main?urlpath=lab)
<div align="center">
<img src="./.github/heart.png" alt="SQL + Python">
</div>
`dask-sql` is a distributed SQL query engine in Python.
It allows you to query and transform your data using a mixture of
common SQL operations and Python code and also scale up the calculation easily
if you need it.
* **Combine the power of Python and SQL**: load your data with Python, transform it with SQL, enhance it with Python and query it with SQL - or the other way round.
With `dask-sql` you can mix the well known Python dataframe API of `pandas` and `Dask` with common SQL operations, to
process your data in exactly the way that is easiest for you.
* **Infinite Scaling**: using the power of the great `Dask` ecosystem, your computations can scale as you need it - from your laptop to your super cluster - without changing any line of SQL code. From k8s to cloud deployments, from batch systems to YARN - if `Dask` [supports it](https://docs.dask.org/en/latest/setup.html), so will `dask-sql`.
* **Your data - your queries**: Use Python user-defined functions (UDFs) in SQL without any performance drawback and extend your SQL queries with the large number of Python libraries, e.g. machine learning, different complicated input formats, complex statistics.
* **Easy to install and maintain**: `dask-sql` is just a pip/conda install away (or a docker run if you prefer).
* **Use SQL from wherever you like**: `dask-sql` integrates with your jupyter notebook, your normal Python module or can be used as a standalone SQL server from any BI tool. It even integrates natively with [Apache Hue](https://gethue.com/).
* **GPU Support**: `dask-sql` supports running SQL queries on CUDA-enabled GPUs by utilizing [RAPIDS](https://rapids.ai) libraries like [`cuDF`](https://github.com/rapidsai/cudf), enabling accelerated compute for SQL.
Read more in the [documentation](https://dask-sql.readthedocs.io/en/latest/).
<div align="center">
<img src="./.github/animation.gif" alt="dask-sql GIF">
</div>
---
## Example
For this example, we use some data loaded from disk and query them with a SQL command from our python code.
Any pandas or dask dataframe can be used as input and ``dask-sql`` understands a large amount of formats (csv, parquet, json,...) and locations (s3, hdfs, gcs,...).
```python
import dask.dataframe as dd
from dask_sql import Context
# Create a context to hold the registered tables
c = Context()
# Load the data and register it in the context
# This will give the table a name, that we can use in queries
df = dd.read_csv("...")
c.create_table("my_data", df)
# Now execute a SQL query. The result is again dask dataframe.
result = c.sql("""
SELECT
my_data.name,
SUM(my_data.x)
FROM
my_data
GROUP BY
my_data.name
""", return_futures=False)
# Show the result
print(result)
```
## Quickstart
Have a look into the [documentation](https://dask-sql.readthedocs.io/en/latest/) or start the example notebook on [binder](https://mybinder.org/v2/gh/dask-contrib/dask-sql-binder/main?urlpath=lab).
> `dask-sql` is currently under development and does so far not understand all SQL commands (but a large fraction).
We are actively looking for feedback, improvements and contributors!
## Installation
`dask-sql` can be installed via `conda` (preferred) or `pip` - or in a development environment.
### With `conda`
Create a new conda environment or use your already present environment:
conda create -n dask-sql
conda activate dask-sql
Install the package from the `conda-forge` channel:
conda install dask-sql -c conda-forge
### With `pip`
You can install the package with
pip install dask-sql
### For development
If you want to have the newest (unreleased) `dask-sql` version or if you plan to do development on `dask-sql`, you can also install the package from sources.
git clone https://github.com/dask-contrib/dask-sql.git
Create a new conda environment and install the development environment:
conda env create -f continuous_integration/environment-3.9.yaml
It is not recommended to use `pip` instead of `conda` for the environment setup.
After that, you can install the package in development mode
pip install -e ".[dev]"
The Rust DataFusion bindings are built as part of the `pip install`.
Note that if changes are made to the Rust source in `src/`, another build must be run to recompile the bindings.
This repository uses [pre-commit](https://pre-commit.com/) hooks. To install them, call
pre-commit install
## Testing
You can run the tests (after installation) with
pytest tests
GPU-specific tests require additional dependencies specified in `continuous_integration/gpuci/environment.yaml`.
These can be added to the development environment by running
```
conda env update -n dask-sql -f continuous_integration/gpuci/environment.yaml
```
And GPU-specific tests can be run with
```
pytest tests -m gpu --rungpu
```
## SQL Server
`dask-sql` comes with a small test implementation for a SQL server.
Instead of rebuilding a full ODBC driver, we re-use the [presto wire protocol](https://github.com/prestodb/presto/wiki/HTTP-Protocol).
It is - so far - only a start of the development and missing important concepts, such as
authentication.
You can test the sql presto server by running (after installation)
dask-sql-server
or by using the created docker image
docker run --rm -it -p 8080:8080 nbraun/dask-sql
in one terminal. This will spin up a server on port 8080 (by default)
that looks similar to a normal presto database to any presto client.
You can test this for example with the default [presto client](https://prestosql.io/docs/current/installation/cli.html):
presto --server localhost:8080
Now you can fire simple SQL queries (as no data is loaded by default):
=> SELECT 1 + 1;
EXPR$0
--------
2
(1 row)
You can find more information in the [documentation](https://dask-sql.readthedocs.io/en/latest/pages/server.html).
## CLI
You can also run the CLI `dask-sql` for testing out SQL commands quickly:
dask-sql --load-test-data --startup
(dask-sql) > SELECT * FROM timeseries LIMIT 10;
## How does it work?
At the core, `dask-sql` does two things:
- translate the SQL query using [DataFusion](https://arrow.apache.org/datafusion) into a relational algebra, which is represented as a logical query plan - similar to many other SQL engines (Hive, Flink, ...)
- convert this description of the query into dask API calls (and execute them) - returning a dask dataframe.
For the first step, Arrow DataFusion needs to know about the columns and types of the dask dataframes, therefore some Rust code to store this information for dask dataframes are defined in `dask_planner`.
After the translation to a relational algebra is done (using `DaskSQLContext.logical_relational_algebra`), the python methods defined in `dask_sql.physical` turn this into a physical dask execution plan by converting each piece of the relational algebra one-by-one.
================================================
FILE: conftest.py
================================================
import dask
import pytest
pytest_plugins = ["tests.integration.fixtures"]
def pytest_addoption(parser):
parser.addoption("--rungpu", action="store_true", help="run tests meant for GPU")
parser.addoption("--runqueries", action="store_true", help="run test queries")
parser.addoption("--data_dir", help="specify file path to the data")
parser.addoption("--queries_dir", help="specify file path to the queries")
def pytest_runtest_setup(item):
# TODO: get pyarrow strings and p2p shuffle working
dask.config.set({"dataframe.convert-string": False})
dask.config.set({"dataframe.shuffle.method": "tasks"})
if "gpu" in item.keywords:
if not item.config.getoption("--rungpu"):
pytest.skip("need --rungpu option to run")
# manually enable cudf decimal support
dask.config.set({"sql.mappings.decimal_support": "cudf"})
if "queries" in item.keywords and not item.config.getoption("--runqueries"):
pytest.skip("need --runqueries option to run")
@pytest.fixture(scope="session")
def data_dir(request):
return request.config.getoption("--data_dir")
@pytest.fixture(scope="session")
def queries_dir(request):
return request.config.getoption("--queries_dir")
================================================
FILE: continuous_integration/docker/cloud.dockerfile
================================================
ARG DOCKER_META_VERSION
FROM nbraun/dask-sql:${DOCKER_META_VERSION}
RUN conda config --add channels conda-forge \
&& /opt/conda/bin/mamba install --freeze-installed -y \
s3fs \
dask-cloudprovider \
&& pip install awscli \
&& conda clean -ay
ENTRYPOINT ["tini", "-g", "--", "/usr/bin/prepare.sh"]
================================================
FILE: continuous_integration/docker/conda.txt
================================================
python>=3.9
dask>=2024.4.1
pandas>=1.4.0
jpype1>=1.0.2
openjdk>=8
maven>=3.6.0
pytest>=6.0.2
pytest-cov>=2.10.1
pytest-xdist
mock>=4.0.3
sphinx>=3.2.1
tzlocal>=2.1
fastapi>=0.92.0
httpx>=0.24.1
uvicorn>=0.14
pyarrow>=14.0.1
prompt_toolkit>=3.0.8
pygments>=2.7.1
scikit-learn>=1.0.0
intake>=0.6.0
pre-commit>=2.11.1
black=22.10.0
isort=5.12.0
maturin>=1.3,<1.4
================================================
FILE: continuous_integration/docker/main.dockerfile
================================================
# Dockerfile for dask-sql running the SQL server
# For more information, see https://dask-sql.readthedocs.io/.
FROM daskdev/dask:latest
LABEL author "Nils Braun <nilslennartbraun@gmail.com>"
# Install rustc & gcc for compilation of DataFusion planner
ADD https://sh.rustup.rs /rustup-init.sh
RUN sh /rustup-init.sh -y --default-toolchain=stable --profile=minimal \
&& apt-get update \
&& apt-get install gcc -y
ENV PATH="/root/.cargo/bin:${PATH}"
# Install conda dependencies for dask-sql
COPY continuous_integration/docker/conda.txt /opt/dask_sql/
RUN mamba install -y \
# build requirements
"maturin>=1.3,<1.4" \
# core dependencies
"dask>=2024.4.1" \
"pandas>=1.4.0" \
"fastapi>=0.92.0" \
"httpx>=0.24.1" \
"uvicorn>=0.14" \
"tzlocal>=2.1" \
"prompt_toolkit>=3.0.8" \
"pygments>=2.7.1" \
tabulate \
# additional dependencies
"pyarrow>=14.0.1" \
"scikit-learn>=1.0.0" \
"intake>=0.6.0" \
&& conda clean -ay
# install dask-sql
COPY Cargo.toml /opt/dask_sql/
COPY Cargo.lock /opt/dask_sql/
COPY pyproject.toml /opt/dask_sql/
COPY setup.cfg /opt/dask_sql/
COPY README.md /opt/dask_sql/
COPY .git /opt/dask_sql/.git
COPY src /opt/dask_sql/src
COPY dask_sql /opt/dask_sql/dask_sql
RUN cd /opt/dask_sql/ \
&& CONDA_PREFIX="/opt/conda/" maturin develop
# Set the script to execute
COPY continuous_integration/scripts/startup_script.py /opt/dask_sql/startup_script.py
EXPOSE 8080
ENTRYPOINT [ "/usr/bin/prepare.sh", "/opt/conda/bin/python", "/opt/dask_sql/startup_script.py" ]
================================================
FILE: continuous_integration/environment-3.10.yaml
================================================
name: dask-sql
channels:
- conda-forge
dependencies:
- c-compiler
- dask>=2024.4.1
- dask-expr>=1.0.11
- docker-py>=7.1.0
- fastapi>=0.92.0
- fugue>=0.7.3
- httpx>=0.24.1
- intake>=0.6.0
- jsonschema
- lightgbm
- maturin>=1.3,<1.4
- mlflow>=2.10
- mock
- numpy>=1.22.4
- pandas>=2
- pre-commit
- prompt_toolkit>=3.0.8
- psycopg2
- pyarrow>=14.0.1
- pygments>=2.7.1
- pyhive
- pytest-cov
- pytest-rerunfailures
- pytest-xdist
- pytest
- python=3.10
- py-xgboost>=2.0.3
- scikit-learn>=1.0.0
- sphinx
- sqlalchemy
- tpot>=0.12.0
# FIXME: https://github.com/fugue-project/fugue/issues/526
- triad<0.9.2
- tzlocal>=2.1
- uvicorn>=0.14
- zlib
================================================
FILE: continuous_integration/environment-3.11.yaml
================================================
name: dask-sql
channels:
- conda-forge
dependencies:
- c-compiler
- dask>=2024.4.1
- dask-expr>=1.0.11
- docker-py>=7.1.0
- fastapi>=0.92.0
- fugue>=0.7.3
- httpx>=0.24.1
- intake>=0.6.0
- jsonschema
- lightgbm
- maturin>=1.3,<1.4
- mlflow>=2.10
- mock
- numpy>=1.22.4
- pandas>=2
- pre-commit
- prompt_toolkit>=3.0.8
- psycopg2
- pyarrow>=14.0.1
- pygments>=2.7.1
- pyhive
- pytest-cov
- pytest-rerunfailures
- pytest-xdist
- pytest
- python=3.11
- py-xgboost>=2.0.3
- scikit-learn>=1.0.0
- sphinx
- sqlalchemy
- tpot>=0.12.0
# FIXME: https://github.com/fugue-project/fugue/issues/526
- triad<0.9.2
- tzlocal>=2.1
- uvicorn>=0.14
- zlib
================================================
FILE: continuous_integration/environment-3.12.yaml
================================================
name: dask-sql
channels:
- conda-forge
dependencies:
- c-compiler
- dask>=2024.4.1
- dask-expr>=1.0.11
- docker-py>=7.1.0
- fastapi>=0.92.0
- fugue>=0.7.3
- httpx>=0.24.1
- intake>=0.6.0
- jsonschema
- lightgbm
- maturin>=1.3,<1.4
# TODO: add once mlflow 3.12 builds are available
# - mlflow>=2.10
- mock
- numpy>=1.22.4
- pandas>=2
- pre-commit
- prompt_toolkit>=3.0.8
- psycopg2
- pyarrow>=14.0.1
- pygments>=2.7.1
- pyhive
- pytest-cov
- pytest-rerunfailures
- pytest-xdist
- pytest
- python=3.12
- py-xgboost>=2.0.3
- scikit-learn>=1.0.0
- sphinx
- sqlalchemy
# TODO: add once tpot supports python 3.12
# - tpot>=0.12.0
# FIXME: https://github.com/fugue-project/fugue/issues/526
- triad<0.9.2
- tzlocal>=2.1
- uvicorn>=0.14
- zlib
================================================
FILE: continuous_integration/environment-3.9.yaml
================================================
name: dask-sql-py39
channels:
- conda-forge
dependencies:
- c-compiler
- dask=2024.4.1
- dask-expr=1.0.11
- docker-py>=7.1.0
- fastapi=0.92.0
- fugue=0.7.3
- httpx=0.24.1
- intake=0.6.0
- jsonschema
- lightgbm
- maturin=1.3
- mlflow=2.10
- mock
- numpy=1.22.4
- pandas=2
- pre-commit
- prompt_toolkit=3.0.8
- psycopg2
- pyarrow=14.0.1
- pygments=2.7.1
- pyhive
- pytest-cov
- pytest-rerunfailures
- pytest-xdist
- pytest
- python=3.9
- py-xgboost=2.0.3
- scikit-learn=1.0.0
- sphinx
- sqlalchemy
- tpot>=0.12.0
# FIXME: https://github.com/fugue-project/fugue/issues/526
- triad<0.9.2
- tzlocal=2.1
- uvicorn=0.14
- zlib
================================================
FILE: continuous_integration/gpuci/environment-3.10.yaml
================================================
name: dask-sql
channels:
- rapidsai
- rapidsai-nightly
- dask/label/dev
- conda-forge
- nvidia
- nodefaults
dependencies:
- c-compiler
- zlib
- dask>=2024.4.1
- dask-expr>=1.0.11
- fastapi>=0.92.0
- fugue>=0.7.3
- httpx>=0.24.1
- intake>=0.6.0
- jsonschema
- lightgbm
- maturin>=1.3,<1.4
- mock
- numpy>=1.22.4
- pandas>=2
- pre-commit
- prompt_toolkit>=3.0.8
- psycopg2
- pyarrow>=14.0.1
- pygments>=2.7.1
- pyhive
- pytest-cov
- pytest-rerunfailures
- pytest-xdist
- pytest
- python=3.10
- py-xgboost>=2.0.3
- scikit-learn>=1.0.0
- sphinx
- sqlalchemy
- tpot>=0.12.0
# FIXME: https://github.com/fugue-project/fugue/issues/526
- triad<0.9.2
- tzlocal>=2.1
- uvicorn>=0.14
# GPU-specific requirements
- cudatoolkit=11.8
- cudf=24.06
- cuml=24.06
- dask-cudf=24.06
- dask-cuda=24.06
- ucx-proc=*=gpu
- ucx-py=0.38
- xgboost=*=rapidsai_py*
- libxgboost=*=rapidsai_h*
================================================
FILE: continuous_integration/gpuci/environment-3.11.yaml
================================================
name: dask-sql
channels:
- rapidsai
- rapidsai-nightly
- dask/label/dev
- conda-forge
- nvidia
- nodefaults
dependencies:
- c-compiler
- zlib
- dask>=2024.4.1
- dask-expr>=1.0.11
- fastapi>=0.92.0
- fugue>=0.7.3
- httpx>=0.24.1
- intake>=0.6.0
- jsonschema
- lightgbm
- maturin>=1.3,<1.4
- mock
- numpy>=1.22.4
- pandas>=2
- pre-commit
- prompt_toolkit>=3.0.8
- psycopg2
- pyarrow>=14.0.1
- pygments>=2.7.1
- pyhive
- pytest-cov
- pytest-rerunfailures
- pytest-xdist
- pytest
- python=3.11
- py-xgboost>=2.0.3
- scikit-learn>=1.0.0
- sphinx
- sqlalchemy
- tpot>=0.12.0
# FIXME: https://github.com/fugue-project/fugue/issues/526
- triad<0.9.2
- tzlocal>=2.1
- uvicorn>=0.14
# GPU-specific requirements
- cudatoolkit=11.8
- cudf=24.06
- cuml=24.06
- dask-cudf=24.06
- dask-cuda=24.06
- ucx-proc=*=gpu
- ucx-py=0.38
- xgboost=*=rapidsai_py*
- libxgboost=*=rapidsai_h*
================================================
FILE: continuous_integration/gpuci/environment-3.9.yaml
================================================
name: dask-sql
channels:
- rapidsai
- rapidsai-nightly
- dask/label/dev
- conda-forge
- nvidia
- nodefaults
dependencies:
- c-compiler
- zlib
- dask>=2024.4.1
- dask-expr>=1.0.11
- fastapi>=0.92.0
- fugue>=0.7.3
- httpx>=0.24.1
- intake>=0.6.0
- jsonschema
- lightgbm
- maturin>=1.3,<1.4
- mock
- numpy>=1.22.4
- pandas>=2
- pre-commit
- prompt_toolkit>=3.0.8
- psycopg2
- pyarrow>=14.0.1
- pygments>=2.7.1
- pyhive
- pytest-cov
- pytest-rerunfailures
- pytest-xdist
- pytest
- python=3.9
- py-xgboost==2.0.3
- scikit-learn>=1.0.0
- sphinx
- sqlalchemy
- tpot>=0.12.0
# FIXME: https://github.com/fugue-project/fugue/issues/526
- triad<0.9.2
- tzlocal>=2.1
- uvicorn>=0.14
# GPU-specific requirements
- cudatoolkit=11.8
- cudf=24.06
- cuml=24.06
- dask-cudf=24.06
- dask-cuda=24.06
- ucx-proc=*=gpu
- ucx-py=0.38
- xgboost=*=rapidsai_py*
- libxgboost=*=rapidsai_h*
================================================
FILE: continuous_integration/recipe/build.sh
================================================
#!/bin/bash
set -ex
# See https://github.com/conda-forge/rust-feedstock/blob/master/recipe/build.sh for cc env explanation
if [ "$c_compiler" = gcc ] ; then
case "$target_platform" in
linux-64) rust_env_arch=X86_64_UNKNOWN_LINUX_GNU ;;
linux-aarch64) rust_env_arch=AARCH64_UNKNOWN_LINUX_GNU ;;
linux-ppc64le) rust_env_arch=POWERPC64LE_UNKNOWN_LINUX_GNU ;;
*) echo "unknown target_platform $target_platform" ; exit 1 ;;
esac
export CARGO_TARGET_${rust_env_arch}_LINKER=$CC
fi
declare -a _xtra_maturin_args
mkdir -p $SRC_DIR/.cargo
if [ "$target_platform" = "osx-64" ] ; then
cat <<EOF >> $SRC_DIR/.cargo/config
[target.x86_64-apple-darwin]
linker = "$CC"
rustflags = [
"-C", "link-arg=-undefined",
"-C", "link-arg=dynamic_lookup",
]
EOF
_xtra_maturin_args+=(--target=x86_64-apple-darwin)
elif [ "$target_platform" = "osx-arm64" ] ; then
cat <<EOF >> $SRC_DIR/.cargo/config
# Required for intermediate codegen stuff
[target.x86_64-apple-darwin]
linker = "$CC_FOR_BUILD"
# Required for final binary artifacts for target
[target.aarch64-apple-darwin]
linker = "$CC"
rustflags = [
"-C", "link-arg=-undefined",
"-C", "link-arg=dynamic_lookup",
]
EOF
_xtra_maturin_args+=(--target=aarch64-apple-darwin)
# This variable must be set to the directory containing the target's libpython DSO
export PYO3_CROSS_LIB_DIR=$PREFIX/lib
# xref: https://github.com/PyO3/pyo3/commit/7beb2720
export PYO3_PYTHON_VERSION=${PY_VER}
# xref: https://github.com/conda-forge/python-feedstock/issues/621
sed -i.bak 's,aarch64,arm64,g' $BUILD_PREFIX/venv/lib/os-patch.py
sed -i.bak 's,aarch64,arm64,g' $BUILD_PREFIX/venv/lib/platform-patch.py
fi
maturin build -vv -j "${CPU_COUNT}" --release --strip --manylinux off --interpreter="${PYTHON}" "${_xtra_maturin_args[@]}"
"${PYTHON}" -m pip install $SRC_DIR/target/wheels/dask_sql*.whl --no-deps -vv
================================================
FILE: continuous_integration/recipe/conda_build_config.yaml
================================================
c_compiler:
- gcc
c_compiler_version:
- '12'
rust_compiler:
- rust
rust_compiler_version:
- '1.72'
maturin:
- '1.3'
xz: # [linux64]
- '5' # [linux64]
================================================
FILE: continuous_integration/recipe/meta.yaml
================================================
{% set name = "dask-sql" %}
{% set major_minor_patch = environ.get('GIT_DESCRIBE_TAG', '0.0.0.dev').split('.') %}
{% set new_patch = major_minor_patch[2] | int + 1 %}
{% set version = (major_minor_patch[:2] + [new_patch]) | join('.') + environ.get('VERSION_SUFFIX', '') %}
package:
name: {{ name|lower }}
version: {{ version }}
source:
git_url: ../..
build:
number: {{ GIT_DESCRIBE_NUMBER }}
entry_points:
- dask-sql-server = dask_sql.server.app:main
- dask-sql = dask_sql.cmd:main
string: py{{ python | replace(".", "") }}_{{ GIT_DESCRIBE_HASH }}_{{ GIT_DESCRIBE_NUMBER }}
requirements:
build:
- python # [build_platform != target_platform]
- cross-python_{{ target_platform }} # [build_platform != target_platform]
- maturin # [build_platform != target_platform]
- {{ compiler('c') }}
- {{ compiler('rust') }}
host:
- pip
- python
- maturin
- xz # [linux64]
run:
- python
- dask >=2024.4.1
- pandas >=1.4.0
- fastapi >=0.92.0
- httpx >=0.24.1
- uvicorn >=0.14
- tzlocal >=2.1
- prompt-toolkit >=3.0.8
- pygments >=2.7.1
- tabulate
test:
imports:
- dask_sql
commands:
- pip check
- dask-sql-server --help
- dask-sql --help
requires:
- pip
about:
home: https://github.com/dask-contrib/dask-sql/
summary: SQL query layer for Dask
license: MIT
license_file: LICENSE.txt
================================================
FILE: continuous_integration/recipe/run_test.py
================================================
import dask.dataframe as dd
import pandas as pd
from dask_sql import Context
c = Context()
data = """
name,x
Alice,34
Bob,
"""
df = pd.DataFrame({"name": ["Alice", "Bob", "Chris"] * 100, "x": list(range(300))})
ddf = dd.from_pandas(df, npartitions=10)
# This needs to be temprarily disabled since this query requires features that are not yet implemented
# c.create_table("my_data", ddf)
# got = c.sql(
# """
# SELECT
# my_data.name,
# SUM(my_data.x) AS "S"
# FROM
# my_data
# GROUP BY
# my_data.name
# """
# )
# expect = pd.DataFrame({"name": ["Alice", "Bob", "Chris"], "S": [14850, 14950, 15050]})
# dd.assert_eq(got, expect)
================================================
FILE: continuous_integration/scripts/startup_script.py
================================================
from dask_sql.server.app import main
if __name__ == "__main__":
main()
================================================
FILE: continuous_integration/scripts/update-dependencies.sh
================================================
#!/bin/bash
UPDATE_ALL_CARGO_DEPS="${UPDATE_ALL_CARGO_DEPS:-true}"
# Update datafusion dependencies in the dask-planner to the latest revision from the default branch
sed -i -r 's/^datafusion-([a-z]+).*/datafusion-\1 = { git = "https:\/\/github.com\/apache\/arrow-datafusion-python\/" }/g' Cargo.toml
if [ "$UPDATE_ALL_CARGO_DEPS" = true ] ; then
cargo update
fi
================================================
FILE: dask_sql/__init__.py
================================================
# FIXME: can we modify TLS model of Rust object to avoid aarch64 glibc bug?
# https://github.com/dask-contrib/dask-sql/issues/1169
from . import _datafusion_lib # isort:skip
import importlib.metadata
from dask.config import set
from . import config
from .cmd import cmd_loop
from .context import Context
from .datacontainer import Statistics
from .server.app import run_server
# TODO: get pyarrow strings and p2p shuffle working
set(dataframe__convert_string=False, dataframe__shuffle__method="tasks")
__version__ = importlib.metadata.version(__name__)
__all__ = [__version__, cmd_loop, Context, run_server, Statistics]
================================================
FILE: dask_sql/_compat.py
================================================
import prompt_toolkit
from packaging.version import parse as parseVersion
_prompt_toolkit_version = parseVersion(prompt_toolkit.__version__)
# TODO: remove if prompt-toolkit min version gets bumped
PIPE_INPUT_CONTEXT_MANAGER = _prompt_toolkit_version >= parseVersion("3.0.29")
================================================
FILE: dask_sql/cmd.py
================================================
import logging
import os
import sys
import tempfile
import traceback
from argparse import ArgumentParser
from functools import partial
from typing import Union
import pandas as pd
from dask.datasets import timeseries
from dask.distributed import Client, as_completed
from prompt_toolkit.auto_suggest import AutoSuggestFromHistory
from prompt_toolkit.completion import WordCompleter
from prompt_toolkit.history import FileHistory
from prompt_toolkit.shortcuts import ProgressBar
from pygments.lexers.sql import SqlLexer
try:
# prompt_toolkit version >= 2
from prompt_toolkit.lexers import PygmentsLexer
except ImportError: # pragma: no cover
# prompt_toolkit version < 2
from prompt_toolkit.layout.lexers import PygmentsLexer
from dask_sql.context import Context
meta_command_completer = WordCompleter(
["\\l", "\\d?", "\\dt", "\\df", "\\de", "\\dm", "\\conninfo", "quit"]
)
class CompatiblePromptSession:
"""
Session object wrapper for the prompt_toolkit module
In the version jump from 1 to 2, the prompt_toolkit
introduced a PromptSession object.
Some environments however (e.g. google collab)
still rely on an older prompt_toolkit version,
so we try to support both versions
with this wrapper object.
All it does is export a `prompt` function.
"""
def __init__(self, lexer) -> None: # pragma: no cover
# make sure everytime dask-sql uses same history file
kwargs = {
"lexer": lexer,
"history": FileHistory(
os.path.join(tempfile.gettempdir(), "dask-sql-history")
),
"auto_suggest": AutoSuggestFromHistory(),
"completer": meta_command_completer,
}
try:
# Version >= 2.0.1: we can use the session object
from prompt_toolkit import PromptSession
session = PromptSession(**kwargs)
self.prompt = session.prompt
except ImportError:
# Version < 2.0: there is no session object
from prompt_toolkit.shortcuts import prompt
self.prompt = partial(prompt, **kwargs)
def _display_markdown(content, **kwargs):
df = pd.DataFrame(content, **kwargs)
print(df.to_markdown(tablefmt="fancy_grid"))
def _parse_meta_command(sql):
command, _, arg = sql.partition(" ")
return command, arg.strip()
def _meta_commands(sql: str, context: Context, client: Client) -> Union[bool, Client]:
"""
parses metacommands and prints their result
returns True if meta commands detected
"""
cmd, schema_name = _parse_meta_command(sql)
available_commands = [
["\\l", "List schemas"],
["\\d?, help, ?", "Show available commands"],
["\\conninfo", "Show Dask cluster info"],
["\\dt [schema]", "List tables"],
["\\df [schema]", "List functions"],
["\\dm [schema]", "List models"],
["\\de [schema]", "List experiments"],
["\\dss [schema]", "Switch schema"],
["\\dsc [dask scheduler address]", "Switch Dask cluster"],
["quit", "Quits dask-sql-cli"],
]
if cmd == "\\dsc":
# Switch Dask cluster
_, scheduler_address = _parse_meta_command(sql)
client = Client(scheduler_address)
return client # pragma: no cover
schema_name = schema_name or context.schema_name
if cmd == "\\d?" or cmd == "help" or cmd == "?":
_display_markdown(available_commands, columns=["Commands", "Description"])
elif cmd == "\\l":
_display_markdown(context.schema.keys(), columns=["Schemas"])
elif cmd == "\\dt":
_display_markdown(context.schema[schema_name].tables.keys(), columns=["Tables"])
elif cmd == "\\df":
_display_markdown(
context.schema[schema_name].functions.keys(), columns=["Functions"]
)
elif cmd == "\\de":
_display_markdown(
context.schema[schema_name].experiments.keys(), columns=["Experiments"]
)
elif cmd == "\\dm":
_display_markdown(context.schema[schema_name].models.keys(), columns=["Models"])
elif cmd == "\\conninfo":
cluster_info = [
["Dask scheduler", client.scheduler.__dict__["addr"]],
["Dask dashboard", client.dashboard_link],
["Cluster status", client.status],
["Dask workers", len(client.cluster.workers)],
]
_display_markdown(
cluster_info, columns=["components", "value"]
) # pragma: no cover
elif cmd == "\\dss":
if schema_name in context.schema:
context.schema_name = schema_name
else:
print(f"Schema {schema_name} not available")
elif cmd == "quit":
print("Quitting dask-sql ...")
client.close() # for safer side
sys.exit()
elif cmd.startswith("\\"):
print(
f"The meta command {cmd} not available, please use commands from below list"
)
_display_markdown(available_commands, columns=["Commands", "Description"])
else:
# nothing detected probably not a meta command
return False
return True
def cmd_loop(
context: Context = None,
client: Client = None,
startup=False,
log_level=None,
): # pragma: no cover
"""
Run a REPL for answering SQL queries using ``dask-sql``.
Every SQL expression that ``dask-sql`` understands can be used here.
Args:
context (:obj:`dask_sql.Context`): If set, use this context instead of an empty one.
client (:obj:`dask.distributed.Client`): If set, use this dask client instead of a new one.
startup (:obj:`bool`): Whether to wait until Apache Calcite was loaded
log_level: (:obj:`str`): The log level of the server and dask-sql
Example:
It is possible to run a REPL by using the CLI script in ``dask-sql``
or by calling this function directly in your user code:
.. code-block:: python
from dask_sql import cmd_loop
# Create your pre-filled context
c = Context()
...
cmd_loop(context=c)
Of course, it is also possible to call the usual ``CREATE TABLE``
commands.
"""
pd.set_option("display.max_rows", None)
pd.set_option("display.max_columns", None)
pd.set_option("display.width", None)
pd.set_option("display.max_colwidth", None)
logging.basicConfig(level=log_level)
client = client or Client()
context = context or Context()
if startup:
context.sql("SELECT 1 + 1").compute()
session = CompatiblePromptSession(lexer=PygmentsLexer(SqlLexer))
while True:
try:
text = session.prompt("(dask-sql) > ")
except KeyboardInterrupt:
continue
except EOFError:
break
text = text.rstrip(";").strip()
if not text:
continue
meta_command_detected = _meta_commands(text, context=context, client=client)
if isinstance(meta_command_detected, Client):
client = meta_command_detected
if not meta_command_detected:
try:
df = context.sql(text, return_futures=True)
if df is not None: # some sql commands returns None
df = df.persist()
# Now turn it into a list of futures
futures = client.futures_of(df)
with ProgressBar() as pb:
for _ in pb(
as_completed(futures), total=len(futures), label="Executing"
):
continue
df = df.compute()
print(df.to_markdown(tablefmt="fancy_grid"))
except Exception:
traceback.print_exc()
def main(): # pragma: no cover
parser = ArgumentParser()
parser.add_argument(
"--scheduler-address",
default=None,
help="Connect to this dask scheduler if given",
)
parser.add_argument(
"--log-level",
default=None,
help="Set the log level of the server. Defaults to info.",
choices=["DEBUG", "INFO", "WARNING", "ERROR"],
)
parser.add_argument(
"--load-test-data",
default=False,
action="store_true",
help="Preload some test data.",
)
parser.add_argument(
"--startup",
default=False,
action="store_true",
help="Wait until Apache Calcite was properly loaded",
)
args = parser.parse_args()
client = None
if args.scheduler_address:
client = Client(args.scheduler_address)
context = Context()
if args.load_test_data:
df = timeseries(freq="1d").reset_index(drop=False)
context.create_table("timeseries", df.persist())
cmd_loop(
context=context, client=client, startup=args.startup, log_level=args.log_level
)
if __name__ == "__main__":
main()
================================================
FILE: dask_sql/config.py
================================================
import os
import dask
import yaml
fn = os.path.join(os.path.dirname(__file__), "sql.yaml")
with open(fn) as f:
defaults = yaml.safe_load(f)
dask.config.update_defaults(defaults)
dask.config.ensure_file(source=fn, comment=True)
================================================
FILE: dask_sql/context.py
================================================
import asyncio
import inspect
import logging
from collections import Counter
from typing import Any, Callable, Union
import dask.dataframe as dd
import pandas as pd
from dask import config as dask_config
from dask.base import optimize
from dask.utils_test import hlg_layer
from dask_sql._datafusion_lib import (
DaskSchema,
DaskSQLContext,
DaskSQLOptimizerConfig,
DaskTable,
DFOptimizationException,
DFParsingException,
LogicalPlan,
)
try:
from dask_sql.physical.utils.statistics import parquet_statistics
except ModuleNotFoundError:
parquet_statistics = None
try:
import dask_cuda # noqa: F401
except ImportError: # pragma: no cover
pass
from dask_sql import input_utils
from dask_sql.datacontainer import (
UDF,
DataContainer,
FunctionDescription,
SchemaContainer,
Statistics,
)
from dask_sql.input_utils import InputType, InputUtil
from dask_sql.integrations.ipython import ipython_integration
from dask_sql.mappings import python_to_sql_type
from dask_sql.physical.rel import RelConverter, custom, logical
from dask_sql.physical.rex import RexConverter, core
from dask_sql.utils import ParsingException
logger = logging.getLogger(__name__)
class Context:
"""
Main object to communicate with ``dask_sql``.
It holds a store of all registered data frames (= tables)
and can convert SQL queries to dask data frames.
The tables in these queries are referenced by the name,
which is given when registering a dask dataframe.
Example:
.. code-block:: python
from dask_sql import Context
c = Context()
# Register a table
c.create_table("my_table", df)
# Now execute an SQL query. The result is a dask dataframe
result = c.sql("SELECT a, b FROM my_table")
# Trigger the computation (or use the data frame for something else)
result.compute()
Usually, you will only ever have a single context in your program.
See also:
:func:`sql`
:func:`create_table`
"""
DEFAULT_CATALOG_NAME = "dask_sql"
DEFAULT_SCHEMA_NAME = "root"
def __init__(self, logging_level=logging.INFO):
"""
Create a new context.
"""
# Set the logging level for this SQL context
logging.basicConfig(level=logging_level)
# Name of the root catalog
self.catalog_name = self.DEFAULT_CATALOG_NAME
# Name of the root schema
self.schema_name = self.DEFAULT_SCHEMA_NAME
# All schema information
self.schema = {self.schema_name: SchemaContainer(self.schema_name)}
# A started SQL server (useful for jupyter notebooks)
self.sql_server = None
# Create the `DaskSQLOptimizerConfig` Rust context
optimizer_config = DaskSQLOptimizerConfig(
dask_config.get("sql.dynamic_partition_pruning"),
dask_config.get("sql.fact_dimension_ratio"),
dask_config.get("sql.max_fact_tables"),
dask_config.get("sql.preserve_user_order"),
dask_config.get("sql.filter_selectivity"),
)
# Create the `DaskSQLContext` Rust context
self.context = DaskSQLContext(
self.catalog_name, self.schema_name, optimizer_config
)
self.context.register_schema(self.schema_name, DaskSchema(self.schema_name))
# # Register any default plugins, if nothing was registered before.
RelConverter.add_plugin_class(logical.DaskAggregatePlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskCrossJoinPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskEmptyRelationPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskFilterPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskJoinPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskLimitPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskProjectPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskSortPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskTableScanPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskUnionPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskValuesPlugin, replace=False)
RelConverter.add_plugin_class(logical.DaskWindowPlugin, replace=False)
RelConverter.add_plugin_class(logical.SamplePlugin, replace=False)
RelConverter.add_plugin_class(logical.ExplainPlugin, replace=False)
RelConverter.add_plugin_class(logical.SubqueryAlias, replace=False)
RelConverter.add_plugin_class(custom.AnalyzeTablePlugin, replace=False)
RelConverter.add_plugin_class(custom.CreateExperimentPlugin, replace=False)
RelConverter.add_plugin_class(custom.CreateModelPlugin, replace=False)
RelConverter.add_plugin_class(custom.CreateCatalogSchemaPlugin, replace=False)
RelConverter.add_plugin_class(custom.CreateMemoryTablePlugin, replace=False)
RelConverter.add_plugin_class(custom.CreateTablePlugin, replace=False)
RelConverter.add_plugin_class(custom.DropModelPlugin, replace=False)
RelConverter.add_plugin_class(custom.DropSchemaPlugin, replace=False)
RelConverter.add_plugin_class(custom.DropTablePlugin, replace=False)
RelConverter.add_plugin_class(custom.ExportModelPlugin, replace=False)
RelConverter.add_plugin_class(custom.PredictModelPlugin, replace=False)
RelConverter.add_plugin_class(custom.ShowColumnsPlugin, replace=False)
RelConverter.add_plugin_class(custom.DescribeModelPlugin, replace=False)
RelConverter.add_plugin_class(custom.ShowModelsPlugin, replace=False)
RelConverter.add_plugin_class(custom.ShowSchemasPlugin, replace=False)
RelConverter.add_plugin_class(custom.ShowTablesPlugin, replace=False)
RelConverter.add_plugin_class(custom.UseSchemaPlugin, replace=False)
RelConverter.add_plugin_class(custom.AlterSchemaPlugin, replace=False)
RelConverter.add_plugin_class(custom.AlterTablePlugin, replace=False)
RelConverter.add_plugin_class(custom.DistributeByPlugin, replace=False)
RexConverter.add_plugin_class(core.RexAliasPlugin, replace=False)
RexConverter.add_plugin_class(core.RexCallPlugin, replace=False)
RexConverter.add_plugin_class(core.RexInputRefPlugin, replace=False)
RexConverter.add_plugin_class(core.RexLiteralPlugin, replace=False)
RexConverter.add_plugin_class(core.RexScalarSubqueryPlugin, replace=False)
InputUtil.add_plugin_class(input_utils.DaskInputPlugin, replace=False)
InputUtil.add_plugin_class(input_utils.PandasLikeInputPlugin, replace=False)
InputUtil.add_plugin_class(input_utils.HiveInputPlugin, replace=False)
InputUtil.add_plugin_class(input_utils.IntakeCatalogInputPlugin, replace=False)
InputUtil.add_plugin_class(input_utils.SqlalchemyHiveInputPlugin, replace=False)
# needs to be the last entry, as it only checks for string
InputUtil.add_plugin_class(input_utils.LocationInputPlugin, replace=False)
def create_table(
self,
table_name: str,
input_table: InputType,
format: str = None,
persist: bool = False,
schema_name: str = None,
statistics: Statistics = None,
gpu: bool = False,
**kwargs,
):
"""
Registering a (dask/pandas) table makes it usable in SQL queries.
The name you give here can be used as table name in the SQL later.
Please note, that the table is stored as it is now.
If you change the table later, you need to re-register.
Instead of passing an already loaded table, it is also possible
to pass a string to a storage location.
The library will then try to load the data using one of
`dask's read methods <https://docs.dask.org/en/latest/dataframe-create.html>`_.
If the file format can not be deduced automatically, it is also
possible to specify it via the ``format`` parameter.
Typical file formats are csv or parquet.
Any additional parameters will get passed on to the read method.
Please note that some file formats require additional libraries.
By default, the data will be lazily loaded. If you would like to
load the data directly into memory you can do so by setting
persist=True.
See :ref:`data_input` for more information.
Example:
This code registers a data frame as table "data"
and then uses it in a query.
.. code-block:: python
c.create_table("data", df)
df_result = c.sql("SELECT a, b FROM data")
This code reads a file from disk.
Please note that we assume that the file(s) are reachable under this path
from every node in the cluster
.. code-block:: python
c.create_table("data", "/home/user/data.csv")
df_result = c.sql("SELECT a, b FROM data")
This example reads from a hive table.
.. code-block:: python
from pyhive.hive import connect
cursor = connect("localhost", 10000).cursor()
c.create_table("data", cursor, hive_table_name="the_name_in_hive")
df_result = c.sql("SELECT a, b FROM data")
Args:
table_name: (:obj:`str`): Under which name should the new table be addressable
input_table (:class:`dask.dataframe.DataFrame` or :class:`pandas.DataFrame` or :obj:`str` or :class:`hive.Cursor`):
The data frame/location/hive connection to register.
format (:obj:`str`): Only used when passing a string into the ``input`` parameter.
Specify the file format directly here if it can not be deduced from the extension.
If set to "memory", load the data from a published dataset in the dask cluster.
persist (:obj:`bool`): Only used when passing a string into the ``input`` parameter.
Set to true to turn on loading the file data directly into memory.
schema_name: (:obj:`str`): in which schema to create the table. By default, will use the currently selected schema.
statistics: (:obj:`Statistics`): if given, use these statistics during the cost-based optimization.
gpu: (:obj:`bool`): if set to true, use dask-cudf to run the data frame calculations on your GPU.
Please note that the GPU support is currently not covering all of dask-sql's SQL language.
**kwargs: Additional arguments for specific formats. See :ref:`data_input` for more information.
"""
logger.debug(
f"Creating table: '{table_name}' of format type '{format}' in schema '{schema_name}'"
)
schema_name = schema_name or self.schema_name
dc = InputUtil.to_dc(
input_table,
table_name=table_name,
format=format,
persist=persist,
gpu=gpu,
**kwargs,
)
if type(input_table) == str:
dc.filepath = input_table
self.schema[schema_name].filepaths[table_name.lower()] = input_table
elif hasattr(input_table, "dask") and dd.utils.is_dataframe_like(input_table):
try:
if dd._dask_expr_enabled():
from dask_expr.io.parquet import ReadParquet
dask_filepath = None
operations = input_table.find_operations(ReadParquet)
for op in operations:
dask_filepath = op._args[0]
else:
dask_filepath = hlg_layer(
input_table.dask, "read-parquet"
).creation_info["args"][0]
dc.filepath = dask_filepath
self.schema[schema_name].filepaths[table_name.lower()] = dask_filepath
except KeyError:
logger.debug("Expected 'read-parquet' layer")
if parquet_statistics and not dd._dask_expr_enabled() and not statistics:
statistics = parquet_statistics(dc.df)
if statistics:
row_count = 0
for d in statistics:
row_count += d["num-rows"]
statistics = Statistics(row_count)
if not statistics:
statistics = Statistics(float("nan"))
dc.statistics = statistics
self.schema[schema_name].tables[table_name.lower()] = dc
self.schema[schema_name].statistics[table_name.lower()] = statistics
def drop_table(self, table_name: str, schema_name: str = None):
"""
Remove a table with the given name from the registered tables.
This will also delete the dataframe.
Args:
table_name: (:obj:`str`): Which table to remove.
"""
schema_name = schema_name or self.schema_name
del self.schema[schema_name].tables[table_name]
def drop_schema(self, schema_name: str):
"""
Remove a schema with the given name from the registered schemas.
This will also delete all tables, functions etc.
Args:
schema_name: (:obj:`str`): Which schema to remove.
"""
if schema_name == self.DEFAULT_SCHEMA_NAME:
raise RuntimeError(f"Default Schema `{schema_name}` cannot be deleted")
del self.schema[schema_name]
if self.schema_name == schema_name:
self.schema_name = self.DEFAULT_SCHEMA_NAME
def register_function(
self,
f: Callable,
name: str,
parameters: list[tuple[str, type]],
return_type: type,
replace: bool = False,
schema_name: str = None,
row_udf: bool = False,
):
"""
Register a custom function with the given name.
The function can be used (with this name)
in every SQL queries from now on - but only for scalar operations
(no aggregations).
This means, if you register a function "f", you can now call
.. code-block:: sql
SELECT f(x)
FROM df
Please keep in mind that you can only have one function with the same name,
regardless of whether it is an aggregation or a scalar function. By default,
attempting to register two functions with the same name will raise an error;
setting `replace=True` will give precedence to the most recently registered
function.
For the registration, you need to supply both the
list of parameter and parameter types as well as the
return type. Use `numpy dtypes <https://numpy.org/doc/stable/reference/arrays.dtypes.html>`_ if possible.
More information: :ref:`custom`
Example:
This example registers a function "f", which
calculates the square of an integer and applies
it to the column ``x``.
.. code-block:: python
def f(x):
return x ** 2
c.register_function(f, "f", [("x", np.int64)], np.int64)
sql = "SELECT f(x) FROM df"
df_result = c.sql(sql)
Example of overwriting two functions with the same name:
This example registers a different function "f", which
calculates the floor division of an integer and applies
it to the column ``x``. It also shows how to overwrite
the previous function with the replace parameter.
.. code-block:: python
def f(x):
return x // 2
c.register_function(f, "f", [("x", np.int64)], np.int64, replace=True)
sql = "SELECT f(x) FROM df"
df_result = c.sql(sql)
Args:
f (:obj:`Callable`): The function to register
name (:obj:`str`): Under which name should the new function be addressable in SQL
parameters (:obj:`List[Tuple[str, type]]`): A list ot tuples of parameter name and parameter type.
Use `numpy dtypes <https://numpy.org/doc/stable/reference/arrays.dtypes.html>`_ if possible. This
function is sensitive to the order of specified parameters when `row_udf=True`, and it is assumed
that column arguments are specified in order, followed by scalar arguments.
return_type (:obj:`type`): The return type of the function
replace (:obj:`bool`): If `True`, do not raise an error if a function with the same name is already
present; instead, replace the original function. Default is `False`.
See also:
:func:`register_aggregation`
"""
self._register_callable(
f,
name,
aggregation=False,
parameters=parameters,
return_type=return_type,
replace=replace,
schema_name=schema_name,
row_udf=row_udf,
)
def register_aggregation(
self,
f: dd.Aggregation,
name: str,
parameters: list[tuple[str, type]],
return_type: type,
replace: bool = False,
schema_name: str = None,
):
"""
Register a custom aggregation with the given name.
The aggregation can be used (with this name)
in every SQL queries from now on - but only for aggregation operations
(no scalar function calls).
This means, if you register a aggregation "fagg", you can now call
.. code-block:: sql
SELECT fagg(y)
FROM df
GROUP BY x
Please note that you can always only have one function with the same name;
no matter if it is an aggregation or scalar function.
For the registration, you need to supply both the
list of parameter and parameter types as well as the
return type. Use `numpy dtypes <https://numpy.org/doc/stable/reference/arrays.dtypes.html>`_ if possible.
More information: :ref:`custom`
Example:
The following code registers a new aggregation "fagg", which
computes the sum of a column and uses it on the ``y`` column.
.. code-block:: python
fagg = dd.Aggregation("fagg", lambda x: x.sum(), lambda x: x.sum())
c.register_aggregation(fagg, "fagg", [("x", np.float64)], np.float64)
sql = "SELECT fagg(y) FROM df GROUP BY x"
df_result = c.sql(sql)
Args:
f (:class:`dask.dataframe.Aggregate`): The aggregate to register. See
`the dask documentation <https://docs.dask.org/en/latest/dataframe-groupby.html#aggregate>`_
for more information.
name (:obj:`str`): Under which name should the new aggregate be addressable in SQL
parameters (:obj:`List[Tuple[str, type]]`): A list ot tuples of parameter name and parameter type.
Use `numpy dtypes <https://numpy.org/doc/stable/reference/arrays.dtypes.html>`_ if possible.
return_type (:obj:`type`): The return type of the function
replace (:obj:`bool`): Do not raise an error if the function is already present
See also:
:func:`register_function`
"""
self._register_callable(
f,
name,
aggregation=True,
parameters=parameters,
return_type=return_type,
replace=replace,
schema_name=schema_name,
)
def sql(
self,
sql: Any,
return_futures: bool = True,
dataframes: dict[str, Union[dd.DataFrame, pd.DataFrame]] = None,
gpu: bool = False,
config_options: dict[str, Any] = None,
) -> Union[dd.DataFrame, pd.DataFrame]:
"""
Query the registered tables with the given SQL.
The SQL follows approximately the postgreSQL standard - however, not all
operations are already implemented.
In general, only select statements (no data manipulation) works.
For more information, see :ref:`sql`.
Example:
In this example, a query is called
using the registered tables and then
executed using dask.
.. code-block:: python
result = c.sql("SELECT a, b FROM my_table")
print(result.compute())
Args:
sql (:obj:`str`): The query string to execute
return_futures (:obj:`bool`): Return the unexecuted dask dataframe or the data itself.
Defaults to returning the dask dataframe.
dataframes (:obj:`Dict[str, dask.dataframe.DataFrame]`): additional Dask or pandas dataframes
to register before executing this query
gpu (:obj:`bool`): Whether or not to load the additional Dask or pandas dataframes (if any) on GPU;
requires cuDF / dask-cuDF if enabled. Defaults to False.
config_options (:obj:`Dict[str,Any]`): Specific configuration options to pass during
query execution
Returns:
:obj:`dask.dataframe.DataFrame`: the created data frame of this query.
"""
with dask_config.set(config_options):
if dataframes is not None:
for df_name, df in dataframes.items():
self.create_table(df_name, df, gpu=gpu)
if isinstance(sql, str):
rel, _ = self._get_ral(sql)
elif isinstance(sql, LogicalPlan):
rel = sql
else:
raise RuntimeError(
f"Encountered unsupported `LogicalPlan` sql type: {type(sql)}"
)
return self._compute_table_from_rel(rel, return_futures)
def explain(
self,
sql: str,
dataframes: dict[str, Union[dd.DataFrame, pd.DataFrame]] = None,
gpu: bool = False,
) -> str:
"""
Return the stringified relational algebra that this query will produce
once triggered (with ``sql()``).
Helpful to understand the inner workings of dask-sql, but typically not
needed to query your data.
If the query is of DDL type (e.g. CREATE TABLE or DESCRIBE SCHEMA),
no relational algebra plan is created and therefore nothing returned.
Args:
sql (:obj:`str`): The query string to use
dataframes (:obj:`Dict[str, dask.dataframe.DataFrame]`): additional Dask or pandas dataframes
to register before executing this query
gpu (:obj:`bool`): Whether or not to load the additional Dask or pandas dataframes (if any) on GPU;
requires cuDF / dask-cuDF if enabled. Defaults to False.
Returns:
:obj:`str`: a description of the created relational algebra.
"""
dynamic_partition_pruning = dask_config.get("sql.dynamic_partition_pruning")
if not dask_config.get("sql.optimizer.verbose"):
dask_config.set({"sql.dynamic_partition_pruning": False})
if dataframes is not None:
for df_name, df in dataframes.items():
self.create_table(df_name, df, gpu=gpu)
_, rel_string = self._get_ral(sql)
dask_config.set({"sql.dynamic_partition_pruning": dynamic_partition_pruning})
return rel_string
def visualize(self, sql: str, filename="mydask.png") -> None: # pragma: no cover
"""Visualize the computation of the given SQL into the png"""
result = self.sql(sql, return_futures=True)
(result,) = optimize(result)
result.visualize(filename)
def create_schema(self, schema_name: str):
"""
Create a new schema in the database.
Args:
schema_name (:obj:`str`): The name of the schema to create
"""
self.schema[schema_name] = SchemaContainer(schema_name)
def alter_schema(self, old_schema_name, new_schema_name):
"""
Alter schema
Args:
old_schema_name:
new_schema_name:
"""
self.schema[new_schema_name] = self.schema.pop(old_schema_name)
def alter_table(self, old_table_name, new_table_name, schema_name=None):
"""
Alter Table
Args:
old_table_name:
new_table_name:
schema_name:
"""
if schema_name is None:
schema_name = self.schema_name
self.schema[schema_name].tables[new_table_name] = self.schema[
schema_name
].tables.pop(old_table_name)
def register_experiment(
self,
experiment_name: str,
experiment_results: pd.DataFrame,
schema_name: str = None,
):
schema_name = schema_name or self.schema_name
self.schema[schema_name].experiments[
experiment_name.lower()
] = experiment_results
def register_model(
self,
model_name: str,
model: Any,
training_columns: list[str],
schema_name: str = None,
):
"""
Add a model to the model registry.
A model can be anything which has a `.predict` function that transforms
a Dask dataframe into predicted labels (as a Dask series).
After model registration, the model can be used in calls to
`SELECT ... FROM PREDICT` with the given name.
Instead of creating your own model and register it, you can also
train a model directly in dask-sql. See the SQL command `CrEATE MODEL`.
Args:
model_name (:obj:`str`): The name of the model
model: The model to store
training_columns: (list of str): The names of the columns which were
used during the training.
"""
schema_name = schema_name or self.schema_name
self.schema[schema_name].models[model_name.lower()] = (model, training_columns)
def ipython_magic(
self, auto_include=False, disable_highlighting=True
): # pragma: no cover
"""
Register a new ipython/jupyter magic function "sql"
which sends its input as string to the :func:`sql` function.
After calling this magic function in a Jupyter notebook or
an IPython shell, you can write
.. code-block:: python
%sql SELECT * from data
or
.. code-block:: python
%%sql
SELECT * from data
instead of
.. code-block:: python
c.sql("SELECT * from data")
Args:
auto_include (:obj:`bool`): If set to true, automatically
create a table for every pandas or Dask dataframe in the calling
context. That means, if you define a dataframe in your jupyter
notebook you can use it with the same name in your sql call.
Use this setting with care as any defined dataframe can
easily override tables created via `CREATE TABLE`.
.. code-block:: python
df = ...
# Later, without any calls to create_table
%%sql
SELECT * FROM df
disable_highlighting (:obj:`bool`): If set to true, automatically
disable syntax highlighting. If you are working in jupyter lab,
diable_highlighting must be set to true to enable ipython_magic
functionality. If you are working in a classic jupyter notebook,
you may set disable_highlighting=False if desired.
"""
ipython_integration(
self, auto_include=auto_include, disable_highlighting=disable_highlighting
)
def run_server(self, **kwargs): # pragma: no cover
"""
Run a HTTP server for answering SQL queries using ``dask-sql``.
See :ref:`server` for more information.
Args:
client (:obj:`dask.distributed.Client`): If set, use this dask client instead of a new one.
host (:obj:`str`): The host interface to listen on (defaults to all interfaces)
port (:obj:`int`): The port to listen on (defaults to 8080)
log_level: (:obj:`str`): The log level of the server and dask-sql
"""
from dask_sql.server.app import run_server
self.stop_server()
self.server = run_server(**kwargs)
def stop_server(self): # pragma: no cover
"""
Stop a SQL server started by ``run_server``.
"""
if self.sql_server is not None:
loop = asyncio.get_event_loop()
assert loop
loop.create_task(self.sql_server.shutdown())
self.sql_server = None
def fqn(self, tbl: "DaskTable") -> tuple[str, str]:
"""
Return the fully qualified name of an object, maybe including the schema name.
Args:
tbl (:obj:`DaskTable`): The Rust DaskTable instance of the view or table.
Returns:
:obj:`tuple` of :obj:`str`: The fully qualified name of the object
"""
schema_name, table_name = tbl.getSchema(), tbl.getTableName()
if schema_name is None or schema_name == "":
schema_name = self.schema_name
return schema_name, table_name
def _prepare_schemas(self):
"""
Create a list of schemas filled with the dataframes
and functions we have currently in our schema list
"""
logger.debug(
f"There are {len(self.schema)} existing schema(s): {self.schema.keys()}"
)
schema_list = []
for schema_name, schema in self.schema.items():
logger.debug(f"Preparing Schema: '{schema_name}'")
rust_schema = DaskSchema(schema_name)
if not schema.tables:
logger.warning("No tables are registered.")
for name, dc in schema.tables.items():
row_count = (
float(schema.statistics[name].row_count)
if name in schema.statistics
else float(0)
)
filepath = schema.filepaths[name] if name in schema.filepaths else None
df = dc.df
columns = df.columns
cc = dc.column_container
if not dask_config.get("sql.identifier.case_sensitive"):
columns = [col.lower() for col in columns]
cc = cc.rename_handle_duplicates(df.columns, columns)
dc.column_container = cc
column_type_mapping = list(
zip(columns, map(python_to_sql_type, df.dtypes))
)
table = DaskTable(
schema_name, name, row_count, column_type_mapping, filepath
)
rust_schema.add_table(table)
if not schema.functions:
logger.debug("No custom functions defined.")
for function_description in schema.function_lists:
name = function_description.name
sql_return_type = function_description.return_type
sql_parameters = function_description.parameters
if function_description.aggregation:
logger.debug(f"Adding function '{name}' to schema as aggregation.")
rust_schema.add_or_overload_function(
name,
[param[1].getDataType() for param in sql_parameters],
sql_return_type.getDataType(),
True,
)
else:
logger.debug(
f"Adding function '{name}' to schema as scalar function."
)
rust_schema.add_or_overload_function(
name,
[param[1].getDataType() for param in sql_parameters],
sql_return_type.getDataType(),
False,
)
schema_list.append(rust_schema)
return schema_list
def _get_ral(self, sql):
"""Helper function to turn the sql query into a relational algebra and resulting column names"""
logger.debug(f"Entering _get_ral('{sql}')")
optimizer_config = DaskSQLOptimizerConfig(
dask_config.get("sql.dynamic_partition_pruning"),
dask_config.get("sql.fact_dimension_ratio"),
dask_config.get("sql.max_fact_tables"),
dask_config.get("sql.preserve_user_order"),
dask_config.get("sql.filter_selectivity"),
)
self.context.set_optimizer_config(optimizer_config)
# get the schema of what we currently have registered
schemas = self._prepare_schemas()
for schema in schemas:
self.context.register_schema(schema.name, schema)
try:
sqlTree = self.context.parse_sql(sql)
except DFParsingException as pe:
raise ParsingException(sql, str(pe))
logger.debug(f"_get_ral -> sqlTree: {sqlTree}")
rel = sqlTree
# TODO: Need to understand if this list here is actually needed? For now just use the first entry.
if len(sqlTree) > 1:
raise RuntimeError(
f"Multiple 'Statements' encountered for SQL {sql}. Please share this with the dev team!"
)
try:
nonOptimizedRel = self.context.logical_relational_algebra(sqlTree[0])
except DFParsingException as pe:
raise ParsingException(sql, str(pe)) from None
# Optimize the `LogicalPlan` or skip if configured
if dask_config.get("sql.optimize"):
try:
rel = self.context.run_preoptimizer(nonOptimizedRel)
rel = self.context.optimize_relational_algebra(rel)
except DFOptimizationException as oe:
# Use original plan and warn about inability to optimize plan
rel = nonOptimizedRel
logger.warning(str(oe))
else:
rel = nonOptimizedRel
rel_string = rel.explain_original()
logger.debug(f"_get_ral -> LogicalPlan: {rel}")
logger.debug(f"Extracted relational algebra:\n {rel_string}")
return rel, rel_string
def _compute_table_from_rel(self, rel: "LogicalPlan", return_futures: bool = True):
dc = RelConverter.convert(rel, context=self)
if rel.get_current_node_type() == "Explain":
return dc
if dc is None:
return
# Optimization might remove some alias projects. Make sure to keep them here.
select_names = [field for field in rel.getRowType().getFieldList()]
if select_names:
cc = dc.column_container
select_names = select_names[: len(cc.columns)]
# Use FQ name if not unique and simple name if it is unique. If a join contains the same column
# names the output col is prepended with the fully qualified column name
field_counts = Counter([field.getName() for field in select_names])
select_names = [
field.getQualifiedName()
if field_counts[field.getName()] > 1
else field.getName()
for field in select_names
]
cc = cc.rename(
{
df_col: select_name
for df_col, select_name in zip(cc.columns, select_names)
}
)
dc = DataContainer(dc.df, cc)
df = dc.assign()
if not return_futures:
df = df.compute()
return df
def _get_tables_from_stack(self):
"""Helper function to return all dask/pandas dataframes from the calling stack"""
stack = inspect.stack()
tables = {}
# Traverse the stacks from inside to outside
for frame_info in stack:
for var_name, variable in frame_info.frame.f_locals.items():
if var_name.startswith("_"):
continue
if not dd.utils.is_dataframe_like(variable):
continue
# only set them if not defined in an inner context
tables[var_name] = tables.get(var_name, variable)
return tables
def _register_callable(
self,
f: Any,
name: str,
aggregation: bool,
parameters: list[tuple[str, type]],
return_type: type,
replace: bool = False,
schema_name=None,
row_udf: bool = False,
):
"""Helper function to do the function or aggregation registration"""
schema_name = schema_name or self.schema_name
schema = self.schema[schema_name]
# validate and cache UDF metadata
sql_parameters = [
(name, python_to_sql_type(param_type)) for name, param_type in parameters
]
sql_return_type = python_to_sql_type(return_type)
if not aggregation:
f = UDF(f, row_udf, parameters, return_type)
lower_name = name.lower()
if lower_name in schema.functions:
if replace:
schema.function_lists = list(
filter(
lambda f: f.name.lower() != lower_name,
schema.function_lists,
)
)
del schema.functions[lower_name]
elif schema.functions[lower_name] != f:
raise ValueError(
"Registering multiple functions with the same name is only permitted if replace=True"
)
schema.function_lists.append(
FunctionDescription(
name.upper(), sql_parameters, sql_return_type, aggregation
)
)
schema.function_lists.append(
FunctionDescription(
name.lower(), sql_parameters, sql_return_type, aggregation
)
)
schema.functions[lower_name] = f
================================================
FILE: dask_sql/datacontainer.py
================================================
from collections import namedtuple
from typing import Any, Union
import dask.dataframe as dd
import pandas as pd
ColumnType = Union[str, int]
FunctionDescription = namedtuple(
"FunctionDescription", ["name", "parameters", "return_type", "aggregation"]
)
class ColumnContainer:
# Forward declaration
pass
class ColumnContainer:
"""
Helper class to store a list of columns,
which do not necessarily be the ones of the dask dataframe.
Instead, the container also stores a mapping from "frontend"
columns (columns with the names and order expected by SQL)
to "backend" columns (the real column names used by dask)
to prevent unnecessary renames.
"""
def __init__(
self,
frontend_columns: list[str],
frontend_backend_mapping: Union[dict[str, ColumnType], None] = None,
):
assert all(
isinstance(col, str) for col in frontend_columns
), "All frontend columns need to be of string type"
self._frontend_columns = list(frontend_columns)
if frontend_backend_mapping is None:
self._frontend_backend_mapping = {
col: col for col in self._frontend_columns
}
else:
self._frontend_backend_mapping = frontend_backend_mapping
def _copy(self) -> ColumnContainer:
"""
Internal function to copy this container
"""
return ColumnContainer(
self._frontend_columns.copy(), self._frontend_backend_mapping.copy()
)
def limit_to(self, fields: list[str]) -> ColumnContainer:
"""
Create a new ColumnContainer, which has frontend columns
limited to only the ones given as parameter.
Also uses the order of these as the new column order.
"""
if not fields:
return self # pragma: no cover
assert all(f in self._frontend_backend_mapping for f in fields)
cc = self._copy()
cc._frontend_columns = [str(x) for x in fields]
return cc
def rename(self, columns: dict[str, str]) -> ColumnContainer:
"""
Return a new ColumnContainer where the frontend columns
are renamed according to the given mapping.
Columns not present in the mapping are not touched,
the order is preserved.
"""
cc = self._copy()
for column_from, column_to in columns.items():
backend_column = self._frontend_backend_mapping[str(column_from)]
cc._frontend_backend_mapping[str(column_to)] = backend_column
cc._frontend_columns = [
str(columns[col]) if col in columns else col
for col in self._frontend_columns
]
return cc
def rename_handle_duplicates(
self, from_columns: list[str], to_columns: list[str]
) -> ColumnContainer:
"""
Same as `rename` but additionally handles presence of
duplicates in `from_columns`
"""
cc = self._copy()
cc._frontend_backend_mapping.update(
{
str(column_to): self._frontend_backend_mapping[str(column_from)]
for column_from, column_to in zip(from_columns, to_columns)
}
)
columns = dict(zip(from_columns, to_columns))
cc._frontend_columns = [
str(columns.get(col, col)) for col in self._frontend_columns
]
return cc
def mapping(self) -> list[tuple[str, ColumnType]]:
"""
The mapping from frontend columns to backend columns.
"""
return list(self._frontend_backend_mapping.items())
@property
def columns(self) -> list[str]:
"""
The stored frontend columns in the correct order
"""
return self._frontend_columns.copy()
def add(
self, frontend_column: str, backend_column: Union[str, None] = None
) -> ColumnContainer:
"""
Return a new ColumnContainer with the
given column added.
The column is added at the last position in the column list.
"""
cc = self._copy()
frontend_column = str(frontend_column)
cc._frontend_backend_mapping[frontend_column] = str(
backend_column or frontend_column
)
if frontend_column not in cc._frontend_columns:
cc._frontend_columns.append(frontend_column)
return cc
def get_backend_by_frontend_index(self, index: int) -> str:
"""
Get back the dask column, which is referenced by the
frontend (SQL) column with the given index.
"""
frontend_column = self._frontend_columns[index]
backend_column = self._frontend_backend_mapping[frontend_column]
return backend_column
def get_backend_by_frontend_name(self, column: str) -> str:
"""
Get back the dask column, which is referenced by the
frontend (SQL) column with the given name.
"""
try:
return self._frontend_backend_mapping[column]
except KeyError:
return column
def make_unique(self, prefix="col"):
"""
Make sure we have unique column names by calling each column
<prefix>_<number>
where <number> is the column index.
"""
return self.rename(
columns={str(col): f"{prefix}_{i}" for i, col in enumerate(self.columns)}
)
class Statistics:
"""
Statistics are used during the cost-based optimization.
Currently, only the row count is supported, more
properties might follow. It needs to be provided by the user.
"""
def __init__(self, row_count: int) -> None:
self.row_count = row_count
def __eq__(self, other):
if isinstance(other, Statistics):
return self.row_count == other.row_count
return False
class DataContainer:
"""
In SQL, every column operation or reference is done via
the column index. Some dask operations, such as grouping,
joining or concatenating preserve the columns in a different
order than SQL would expect.
However, we do not want to change the column data itself
all the time (because this would lead to computational overhead),
but still would like to keep the columns accessible by name and index.
For this, we add an additional `ColumnContainer` to each dataframe,
which does all the column mapping between "frontend"
(what SQL expects, also in the correct order)
and "backend" (what dask has).
"""
def __init__(
self,
df: dd.DataFrame,
column_container: ColumnContainer,
statistics: Statistics = None,
filepath: str = None,
):
self.df = df
self.column_container = column_container
self.statistics = statistics
self.filepath = filepath
def assign(self) -> dd.DataFrame:
"""
Combine the column mapping with the actual data and return
a dataframe which has the the columns specified in the
stored ColumnContainer.
"""
df = self.df[
[
self.column_container._frontend_backend_mapping[out_col]
for out_col in self.column_container.columns
]
]
df.columns = self.column_container.columns
return df
class UDF:
def __init__(self, func, row_udf: bool, params, return_type=None):
"""
Helper class that handles different types of UDFs and manages
how they should be mapped to dask operations. Two versions of
UDFs are supported - when `row_udf=False`, the UDF is treated
as expecting series-like objects as arguments and will simply
run those through the function. When `row_udf=True` a row udf
is expected and should be written to expect a dictlike object
containing scalars
"""
self.row_udf = row_udf
self.func = func
self.names = [param[0] for param in params]
self.meta = (None, return_type)
def __call__(self, *args, **kwargs):
if self.row_udf:
column_args = []
scalar_args = []
for operand in args:
if isinstance(operand, dd.Series):
column_args.append(operand)
else:
scalar_args.append(operand)
df = column_args[0].to_frame(self.names[0])
for name, col in zip(self.names[1:], column_args[1:]):
df[name] = col
result = df.apply(
self.func, axis=1, args=tuple(scalar_args), meta=self.meta
).astype(self.meta[1])
else:
result = self.func(*args, **kwargs)
return result
def __eq__(self, other):
if isinstance(other, UDF):
return self.func == other.func and self.row_udf == other.row_udf
return NotImplemented
def __hash__(self):
return (self.func, self.row_udf).__hash__()
class SchemaContainer:
def __init__(self, name: str):
self.__name__ = name
self.tables: dict[str, DataContainer] = {}
self.statistics: dict[str, Statistics] = {}
self.experiments: dict[str, pd.DataFrame] = {}
self.models: dict[str, tuple[Any, list[str]]] = {}
self.functions: dict[str, UDF] = {}
self.function_lists: list[FunctionDescription] = []
self.filepaths: dict[str, str] = {}
================================================
FILE: dask_sql/input_utils/__init__.py
================================================
from .convert import InputType, InputUtil
from .dask import DaskInputPlugin
from .hive import HiveInputPlugin
from .intake import IntakeCatalogInputPlugin
from .location import LocationInputPlugin
from .pandaslike import PandasLikeInputPlugin
from .sqlalchemy import SqlalchemyHiveInputPlugin
__all__ = [
InputUtil,
InputType,
DaskInputPlugin,
HiveInputPlugin,
IntakeCatalogInputPlugin,
LocationInputPlugin,
PandasLikeInputPlugin,
SqlalchemyHiveInputPlugin,
]
================================================
FILE: dask_sql/input_utils/base.py
================================================
from typing import Any
class BaseInputPlugin:
def is_correct_input(
self, input_item: Any, table_name: str, format: str = None, **kwargs
):
raise NotImplementedError
def to_dc(self, input_item: Any, table_name: str, format: str = None, **kwargs):
raise NotImplementedError
================================================
FILE: dask_sql/input_utils/convert.py
================================================
import logging
from typing import TYPE_CHECKING, Union
import dask.dataframe as dd
import pandas as pd
from dask_sql.datacontainer import ColumnContainer, DataContainer
from dask_sql.input_utils.base import BaseInputPlugin
from dask_sql.utils import Pluggable
if TYPE_CHECKING:
import cudf
import hive
import sqlalchemy
logger = logging.Logger(__name__)
InputType = Union[
dd.DataFrame,
pd.DataFrame,
str,
Union[
"sqlalchemy.engine.base.Connection",
"hive.Cursor",
"cudf.core.dataframe.DataFrame",
],
]
class InputUtil(Pluggable):
"""
Plugin list and helper class for transforming the inputs to
create table into a dask dataframe
"""
@classmethod
def add_plugin_class(cls, plugin_class: BaseInputPlugin, replace=True):
"""Convenience function to add a class directly to the plugins"""
logger.debug(f"Registering Input plugin for {plugin_class}")
cls.add_plugin(str(plugin_class), plugin_class(), replace=replace)
@classmethod
def to_dc(
cls,
input_item: InputType,
table_name: str,
format: str = None,
persist: bool = True,
gpu: bool = False,
**kwargs,
) -> DataContainer:
"""
Turn possible input descriptions or formats (e.g. dask dataframes, pandas dataframes,
locations as string, hive tables) into the loaded data containers,
maybe persist them to cluster memory before.
"""
filled_get_dask_dataframe = lambda *args: cls._get_dask_dataframe(
*args,
table_name=table_name,
format=format,
gpu=gpu,
**kwargs,
)
if isinstance(input_item, list):
table = dd.concat([filled_get_dask_dataframe(item) for item in input_item])
else:
table = filled_get_dask_dataframe(input_item)
if persist:
table = table.persist()
return DataContainer(table.copy(), ColumnContainer(table.columns))
@classmethod
def _get_dask_dataframe(
cls,
input_item: InputType,
table_name: str,
format: str = None,
gpu: bool = False,
**kwargs,
):
plugin_list = cls.get_plugins()
for plugin in plugin_list:
if plugin.is_correct_input(
input_item, table_name=table_name, format=format, **kwargs
):
return plugin.to_dc(
input_item, table_name=table_name, format=format, gpu=gpu, **kwargs
)
raise ValueError(f"Do not understand the input type {type(input_item)}")
================================================
FILE: dask_sql/input_utils/dask.py
================================================
from typing import Any
import dask.dataframe as dd
from dask_sql.input_utils.base import BaseInputPlugin
class DaskInputPlugin(BaseInputPlugin):
"""Input Plugin for Dask DataFrames, just keeping them"""
def is_correct_input(
self, input_item: Any, table_name: str, format: str = None, **kwargs
):
return isinstance(input_item, dd.DataFrame) or format == "dask"
def to_dc(
self,
input_item: Any,
table_name: str,
format: str = None,
gpu: bool = False,
**kwargs
):
if gpu: # pragma: no cover
try:
import dask_cudf # noqa: F401
except ImportError:
raise ModuleNotFoundError(
"Setting `gpu=True` for table creation requires dask_cudf"
)
return input_item.to_backend("cudf", **kwargs)
return input_item
================================================
FILE: dask_sql/input_utils/hive.py
================================================
import ast
import logging
import os
from functools import partial
from typing import Any, Union
import dask.dataframe as dd
from dask_sql._datafusion_lib import SqlTypeName
try:
from pyhive import hive
except ImportError: # pragma: no cover
hive = None
try:
import sqlalchemy
except ImportError: # pragma: no cover
sqlalchemy = None
from dask_sql.input_utils.base import BaseInputPlugin
from dask_sql.mappings import cast_column_type, sql_to_python_type
logger = logging.Logger(__name__)
class HiveInputPlugin(BaseInputPlugin):
"""Input Plugin from Hive"""
def is_correct_input(
self, input_item: Any, table_name: str, format: str = None, **kwargs
):
is_hive_cursor = hive and isinstance(input_item, hive.Cursor)
return self.is_sqlalchemy_hive(input_item) or is_hive_cursor or format == "hive"
def is_sqlalchemy_hive(self, input_item: Any):
return sqlalchemy and isinstance(input_item, sqlalchemy.engine.base.Connection)
def to_dc(
self,
input_item: Any,
table_name: str,
format: str = None,
gpu: bool = False,
**kwargs,
):
if gpu: # pragma: no cover
raise Exception("Hive does not support gpu")
table_name = kwargs.pop("hive_table_name", table_name)
schema = kwargs.pop("hive_schema_name", "default")
parsed = self._parse_hive_table_description(input_item, schema, table_name)
(
column_information,
table_information,
storage_information,
partition_information,
) = parsed
logger.debug("Extracted hive information: ")
logger.debug(f"column information: {column_information}")
logger.debug(f"table information: {table_information}")
logger.debug(f"storage information: {storage_information}")
logger.debug(f"partition information: {partition_information}")
# Convert column information
column_information = {
col: sql_to_python_type(SqlTypeName.fromString(col_type.upper()))
for col, col_type in column_information.items()
}
# Extract format information
if "InputFormat" in storage_information:
format = storage_information["InputFormat"].split(".")[-1]
# databricks format is different, see https://github.com/dask-contrib/dask-sql/issues/83
elif "InputFormat" in table_information: # pragma: no cover
format = table_information["InputFormat"].split(".")[-1]
else: # pragma: no cover
raise RuntimeError(
"Do not understand the output of 'DESCRIBE FORMATTED <table>'"
)
if (
format == "TextInputFormat" or format == "SequenceFileInputFormat"
): # pragma: no cover
storage_description = storage_information.get("Storage Desc Params", {})
read_function = partial(
dd.read_csv,
sep=storage_description.get("field.delim", ","),
header=None,
)
elif format == "ParquetInputFormat" or format == "MapredParquetInputFormat":
read_function = dd.read_parquet
elif format == "OrcInputFormat": # pragma: no cover
read_function = dd.read_orc
elif format == "JsonInputFormat": # pragma: no cover
read_function = dd.read_json
else: # pragma: no cover
raise AttributeError(f"Do not understand hive's table format {format}")
def _normalize(loc):
if loc.startswith("dbfs:/") and not loc.startswith(
"dbfs://"
): # pragma: no cover
# dask (or better: fsspec) needs to have the URL in a specific form
# starting with two // after the protocol
loc = f"dbfs://{loc.lstrip('dbfs:')}"
# file:// is not a known protocol
loc = loc.lstrip("file:")
# Only allow files which do not start with . or _
# Especially, not allow the _SUCCESS files
return os.path.join(loc, "[A-Za-z0-9-]*")
def wrapped_read_function(location, column_information, **kwargs):
location = _normalize(location)
logger.debug(f"Reading in hive data from {location}")
if format == "ParquetInputFormat" or format == "MapredParquetInputFormat":
# Hack needed for parquet files.
# If the folder structure is like .../col=3/...
# parquet wants to read in the partition information.
# However, we add the partition information by ourself
# which will lead to problems afterwards
# Therefore tell parquet to only read in the columns
# we actually care right now
kwargs.setdefault("columns", list(column_information.keys()))
else: # pragma: no cover
# prevent python to optimize it away and make coverage not respect the
# pragma
dummy = 0 # noqa: F841
df = read_function(location, **kwargs)
logger.debug(f"Applying column information: {column_information}")
df = df.rename(columns=dict(zip(df.columns, column_information.keys())))
for col, expected_type in column_information.items():
df = cast_column_type(df, col, expected_type)
return df
if partition_information:
partition_list = self._parse_hive_partition_description(
input_item, schema, table_name
)
logger.debug(f"Reading in partitions from {partition_list}")
tables = []
for partition in partition_list:
parsed = self._parse_hive_table_description(
input_item, schema, table_name, partition=partition
)
(
partition_column_information,
partition_table_information,
_,
_,
) = parsed
location = partition_table_information["Location"]
table = wrapped_read_function(
location, partition_column_information, **kwargs
)
# Now add the additional partition columns
partition_values = ast.literal_eval(
partition_table_information["Partition Value"]
)
# multiple partition column values returned comma separated string
if "," in partition_values:
partition_values = [x.strip() for x in partition_values.split(",")]
logger.debug(
f"Applying additional partition information as columns: {partition_information}"
)
partition_id = 0
for partition_key, partition_type in partition_information.items():
table[partition_key] = partition_values[partition_id]
table = cast_column_type(table, partition_key, partition_type)
partition_id += 1
tables.append(table)
return dd.concat(tables)
location = table_information["Location"]
df = wrapped_read_function(location, column_information, **kwargs)
return df
def _parse_hive_table_description(
self,
cursor: Union["sqlalchemy.engine.base.Connection", "hive.Cursor"],
schema: str,
table_name: str,
partition: str = None,
):
"""
Extract all information from the output
of the DESCRIBE FORMATTED call, which is unfortunately
in a format not easily readable by machines.
"""
cursor.execute(
sqlalchemy.text(f"USE {schema}")
if self.is_sqlalchemy_hive(cursor)
else f"USE {schema}"
)
if partition:
# Hive wants quoted, comma separated list of partition keys
partition = partition.replace("=", '="')
partition = partition.replace("/", '",') + '"'
result = self._fetch_all_results(
cursor, f"DESCRIBE FORMATTED {table_name} PARTITION ({partition})"
)
else:
result = self._fetch_all_results(cursor, f"DESCRIBE FORMATTED {table_name}")
logger.debug(f"Got information from hive: {result}")
table_information = {}
column_information = {} # using the fact that dicts are insertion ordered
storage_information = {}
partition_information = {}
mode = "column"
last_field = None
for key, value, value2 in result:
key = key.strip().rstrip(":") if key else ""
value = value.strip() if value else ""
value2 = value2.strip() if value2 else ""
# That is just a comment line, we can skip it
if key == "# col_name":
continue
if (
key == "# Detailed Table Information"
or key == "# Detailed Partition Information"
):
mode = "table"
elif key == "# Storage Information":
mode = "storage"
elif key == "# Partition Information":
mode = "partition"
elif key.startswith("#"):
mode = None # pragma: no cover
elif key:
if not value:
value = dict()
if mode == "column":
column_information[key] = value
last_field = column_information[key]
elif mode == "storage":
storage_information[key] = value
last_field = storage_information[key]
elif mode == "table":
# Hive partition values come in a bracketed list
# quoted partition values work regardless of partition column type
if key == "Partition Value":
value = '"' + value.strip("[]") + '"'
table_information[key] = value
last_field = table_information[key]
elif mode == "partition":
partition_information[key] = value
last_field = partition_information[key]
else: # pragma: no cover
# prevent python to optimize it away and make coverage not respect the
# pragma
dummy = 0 # noqa: F841
elif value and last_field is not None:
last_field[value] = value2
return (
column_information,
table_information,
storage_information,
partition_information,
)
def _parse_hive_partition_description(
self,
cursor: Union["sqlalchemy.engine.base.Connection", "hive.Cursor"],
schema: str,
table_name: str,
):
"""
Extract all partition informaton for a given table
"""
cursor.execute(
sqlalchemy.text(f"USE {schema}")
if self.is_sqlalchemy_hive(cursor)
else f"USE {schema}"
)
result = self._fetch_all_results(cursor, f"SHOW PARTITIONS {table_name}")
return [row[0] for row in result]
def _fetch_all_results(
self,
cursor: Union["sqlalchemy.engine.base.Connection", "hive.Cursor"],
sql: str,
):
"""
The pyhive.Cursor and the sqlalchemy connection behave slightly different.
The former has the fetchall method on the cursor,
whereas the latter on the executed query.
"""
result = cursor.execute(
sqlalchemy.text(sql) if self.is_sqlalchemy_hive(cursor) else sql
)
try:
return result.fetchall()
except AttributeError: # pragma: no cover
return cursor.fetchall()
================================================
FILE: dask_sql/input_utils/intake.py
================================================
from typing import Any
try:
import intake
except ImportError: # pragma: no cover
intake = None
from dask_sql.input_utils.base import BaseInputPlugin
class IntakeCatalogInputPlugin(BaseInputPlugin):
"""Input Plugin for Intake Catalogs, getting the table in dask format"""
def is_correct_input(
self, input_item: Any, table_name: str, format: str = None, **kwargs
):
return intake and (
isinstance(input_item, intake.catalog.Catalog) or format == "intake"
)
def to_dc(
self,
input_item: Any,
table_name: str,
format: str = None,
gpu: bool = False,
**kwargs,
):
if gpu: # pragma: no cover
raise NotImplementedError("Intake does not support gpu")
table_name = kwargs.pop("intake_table_name", table_name)
catalog_kwargs = kwargs.pop("catalog_kwargs", {})
if isinstance(input_item, str):
input_item = intake.open_catalog(input_item, **catalog_kwargs)
return input_item[table_name].to_dask(**kwargs)
================================================
FILE: dask_sql/input_utils/location.py
================================================
import os
from typing import Any
import dask.dataframe as dd
from distributed.client import default_client
from dask_sql.input_utils.base import BaseInputPlugin
from dask_sql.input_utils.convert import InputUtil
class LocationInputPlugin(BaseInputPlugin):
"""Input Plugin for everything, which can be read in from a file (on disk, remote etc.)"""
def is_correct_input(
self, input_item: Any, table_name: str, format: str = None, **kwargs
):
return isinstance(input_item, str)
def to_dc(
self,
input_item: Any,
table_name: str,
format: str = None,
gpu: bool = False,
**kwargs,
):
if format == "memory":
client = default_client()
df = client.get_dataset(input_item, **kwargs)
plugin_list = InputUtil.get_plugins()
for plugin in plugin_list:
if plugin.is_correct_input(df, table_name, format, **kwargs):
return plugin.to_dc(df, table_name, format, gpu, **kwargs)
if not format:
_, extension = os.path.splitext(input_item)
format = extension.lstrip(".")
try:
if gpu: # pragma: no cover
try:
import dask_cudf
except ImportError:
raise ModuleNotFoundError(
"Setting `gpu=True` for table creation requires dask-cudf"
)
read_function = getattr(dask_cudf, f"read_{format}")
else:
read_function = getattr(dd, f"read_{format}")
except AttributeError:
raise AttributeError(f"Can not read files of format {format}")
return read_function(input_item, **kwargs)
================================================
FILE: dask_sql/input_utils/pandaslike.py
================================================
import dask.dataframe as dd
import pandas as pd
from dask_sql.input_utils.base import BaseInputPlugin
class PandasLikeInputPlugin(BaseInputPlugin):
"""Input Plugin for Pandas Like DataFrames, which get converted to dask DataFrames"""
def is_correct_input(
self, input_item, table_name: str, format: str = None, **kwargs
):
return (
dd.utils.is_dataframe_like(input_item)
and not isinstance(input_item, dd.DataFrame)
) or format == "dask"
def to_dc(
self,
input_item,
table_name: str,
format: str = None,
gpu: bool = False,
**kwargs,
):
npartitions = kwargs.pop("npartitions", 1)
if gpu: # pragma: no cover
try:
import cudf
except ImportError:
raise ModuleNotFoundError(
"Setting `gpu=True` for table creation requires cudf"
)
if isinstance(input_item, pd.DataFrame):
input_item = cudf.from_pandas(input_item)
return dd.from_pandas(input_item, npartitions=npartitions, **kwargs)
================================================
FILE: dask_sql/input_utils/sqlalchemy.py
================================================
from typing import Any
from dask_sql.input_utils.hive import HiveInputPlugin
class SqlalchemyHiveInputPlugin(HiveInputPlugin):
"""Input Plugin from sqlalchemy string"""
def is_correct_input(
self, input_item: Any, table_name: str, format: str = None, **kwargs
):
correct_prefix = isinstance(input_item, str) and (
input_item.startswith("hive://")
or input_item.startswith("databricks+pyhive://")
)
return correct_prefix
def to_dc(
self,
input_item: Any,
table_name: str,
format: str = None,
gpu: bool = False,
**kwargs
): # pragma: no cover
if gpu:
raise NotImplementedError("Hive does not support gpu")
import sqlalchemy
engine_kwargs = {}
if "connect_args" in kwargs:
engine_kwargs["connect_args"] = kwargs.pop("connect_args")
if format is not None:
raise AttributeError(
"Format specified and sqlalchemy connection string set!"
)
cursor = sqlalchemy.create_engine(input_item, **engine_kwargs).connect()
return super().to_dc(cursor, table_name=table_name, **kwargs)
================================================
FILE: dask_sql/integrations/__init__.py
================================================
================================================
FILE: dask_sql/integrations/fugue.py
================================================
try:
import fugue
import fugue_dask
from dask.distributed import Client
from fugue import WorkflowDataFrame, register_execution_engine
from fugue_sql import FugueSQLWorkflow
from triad import run_at_def
from triad.utils.convert import get_caller_global_local_vars
except ImportError: # pragma: no cover
raise ImportError(
"Can not load the fugue module. If you want to use this integration, you need to install it."
)
from typing import Any, Optional
import dask.dataframe as dd
from dask_sql.context import Context
@run_at_def
def _register_engines() -> None:
"""Register (overwrite) the default Dask execution engine of Fugue. This
function is invoked as an entrypoint, users don't need to call it explicitly.
"""
register_execution_engine(
"dask",
lambda conf, **kwargs: DaskSQLExecutionEngine(conf=conf),
on_dup="overwrite",
)
register_execution_engine(
Client,
lambda engine, conf, **kwargs: DaskSQLExecutionEngine(
dask_client=engine, conf=conf
),
on_dup="overwrite",
)
class DaskSQLEngine(fugue.execution.execution_engine.SQLEngine):
"""
SQL engine for fugue which uses dask-sql instead of the native
SQL implementation.
Please note, that so far the native SQL engine in fugue
understands a larger set of SQL commands, but in turns is
(on average) slower in computation and scaling.
"""
def __init__(self, *args, **kwargs):
"""Create a new instance."""
super().__init__(*args, **kwargs)
@property
def is_distributed(self) -> bool:
return True
def select(
self, dfs: fugue.dataframe.DataFrames, statement: str
) -> fugue.dataframe.DataFrame:
"""Send the SQL command to the dask-sql context and register all temporary dataframes"""
c = Context()
for k, v in dfs.items():
c.create_table(k, self.execution_engine.to_df(v).native)
df = c.sql(statement)
return fugue_dask.dataframe.DaskDataFrame(df)
class DaskSQLExecutionEngine(fugue_dask.DaskExecutionEngine):
"""
Execution engine for fugue which has dask-sql as SQL engine
configured.
Please note, that so far the native SQL engine in fugue
understands a larger set of SQL commands, but in turns is
(on average) slower in computation and scaling.
"""
def __init__(self, *args, **kwargs):
"""Create a new instance."""
super().__init__(*args, **kwargs)
self._default_sql_engine = DaskSQLEngine(self)
@property
def default_sql_engine(self) -> fugue.execution.execution_engine.SQLEngine:
return self._default_sql_engine
def fsql_dask(
sql: str,
ctx: Optional[Context] = None,
register: bool = False,
fugue_conf: Any = None,
) -> dict[str, dd.DataFrame]:
"""FugueSQL utility function that can consume Context directly. FugueSQL is a language
extending standard SQL. It makes SQL eligible to describe end to end workflows. It also
enables you to invoke python extensions in the SQL like language.
For more, please read
`FugueSQL Tutorial <https://fugue-tutorials.readthedocs.io/en/latest/tutorials/fugue_sql/index.html/>`_
Args:
sql (:obj:`str`): Fugue SQL statement
ctx (:class:`dask_sql.Context`): The context to operate on, defaults to None
register (:obj:`bool`): Whether to register named steps back to the context
(if provided), defaults to False
fugue_conf (:obj:`Any`): a dictionary like object containing Fugue specific configs
Example:
.. code-block:: python
# define a custom prepartition function for FugueSQL
def median(df: pd.DataFrame) -> pd.DataFrame:
df["y"] = df["y"].median()
return df.head(1)
# create a context with some tables
c = Context()
...
# run a FugueSQL query using the context as input
query = '''
j = SELECT df1.*, df2.x
FROM df1 INNER JOIN df2 ON df1.key = df2.key
PERSIST
TAKE 5 ROWS PREPARTITION BY x PRESORT key
PRINT
TRANSFORM j PREPARTITION BY x USING median
PRINT
'''
result = fsql_dask(query, c, register=True)
assert "j" in result
assert "j" in c.tables
"""
_global, _local = get_caller_global_local_vars()
dag = FugueSQLWorkflow()
dfs = (
{}
if ctx is None
else {k: dag.df(v.df) for k, v in ctx.schema[ctx.schema_name].tables.items()}
)
result = dag._sql(sql, _global, _local, **dfs)
dag.run(DaskSQLExecutionEngine(conf=fugue_conf))
result_dfs = {
k: v.result.native
for k, v in result.items()
if isinstance(v, WorkflowDataFrame)
}
if register and ctx is not None:
for k, v in result_dfs.items():
ctx.create_table(k, v)
return result_dfs
================================================
FILE: dask_sql/integrations/ipython.py
================================================
import time
from typing import TYPE_CHECKING
from dask_sql.mappings import _SQL_TO_PYTHON_FRAMES
from dask_sql.physical.rex.core import RexCallPlugin
if TYPE_CHECKING:
import dask_sql
# That is definitely not pretty, but there seems to be no better way...
KEYWORDS = [
"and",
"as",
"asc",
"between",
"by",
"columns",
"count",
"create",
"delete",
"desc",
"describe",
"distinct",
"exists",
"from",
"group",
"having",
"if",
"in",
"inner",
"insert",
"into",
"is",
"join",
"left",
"like",
"model",
"not",
"on",
"or",
"order",
"outer",
"right",
"schemas",
"select",
"set",
"show",
"table",
"union",
"where",
]
def ipython_integration(
context: "dask_sql.Context",
auto_include: bool,
disable_highlighting: bool,
) -> None: # pragma: no cover
"""Integrate the context with jupyter notebooks. Have a look into :ref:`Context.ipython_magic`."""
_register_ipython_magic(context, auto_include=auto_include)
if not disable_highlighting:
_register_syntax_highlighting()
def _register_ipython_magic(
c: "dask_sql.Context", auto_include: bool
) -> None: # pragma: no cover
from IPython.core.magic import needs_local_scope, register_line_cell_magic
@needs_local_scope
def sql(line, cell, local_ns):
if cell is None:
# the magic function was called inline
cell = line
sql_statement = cell.format(**local_ns)
dataframes = {}
if auto_include:
dataframes = c._get_tables_from_stack()
t0 = time.time()
res = c.sql(sql_statement, return_futures=False, dataframes=dataframes)
if (
"CREATE OR REPLACE TABLE" in sql_statement
or "CREATE OR REPLACE VIEW" in sql_statement
):
table = sql_statement.split("CREATE OR REPLACE")[1]
table = table.replace("TABLE", "").replace("VIEW", "").split()[0].strip()
res = c.sql(f"SELECT * FROM {table}").tail()
elif "CREATE TABLE" in sql_statement or "CREATE VIEW" in sql_statement:
table = sql_statement.split("CREATE")[1]
table = table.replace("TABLE", "").replace("VIEW", "").split()[0].strip()
res = c.sql(f"SELECT * FROM {table}").tail()
print(f"Execution time: {time.time() - t0:.2f}s")
return res
# Register a new magic function
magic_func = register_line_cell_magic(sql)
magic_func.MAGIC_NO_VAR_EXPAND_ATTR = True
def _register_syntax_highlighting(): # pragma: no cover
import json
from IPython.core import display
# JS snippet to use the created mime type highlighthing
_JS_ENABLE_DASK_SQL = r"""
require(['notebook/js/codecell'], function(codecell) {
codecell.CodeCell.options_default.highlight_modes['magic_text/x-dasksql'] = {'reg':[/%%sql/]} ;
Jupyter.notebook.events.on('kernel_ready.Kernel', function(){
Jupyter.notebook.get_cells().map(function(cell){
if (cell.cell_type == 'code'){ cell.auto_highlight(); } }) ;
});
});
"""
types = map(str, _SQL_TO_PYTHON_FRAMES.keys())
functions = list(RexCallPlugin.OPERATION_MAPPING.keys())
# Create a new mimetype
mime_type = {
"name": "sql",
"keywords": _create_set(KEYWORDS + functions),
"builtin": _create_set(types),
"atoms": _create_set(["false", "true", "null"]),
# "operatorChars": /^[*\/+\-%<>!=~&|^]/,
"dateSQL": _create_set(["time"]),
# More information
# https://opensource.apple.com/source/WebInspectorUI/WebInspectorUI-7600.8.3/UserInterface/External/CodeMirror/sql.js.auto.html
"support": _create_set(["ODBCdotTable", "doubleQuote", "zerolessFloat"]),
}
# Code original from fugue-sql, adjusted for dask-sql and using some more customizations
js = (
r"""
require(["codemirror/lib/codemirror"]);
// We define a new mime type for syntax highlighting
CodeMirror.defineMIME("text/x-dasksql", """
+ json.dumps(mime_type)
+ r"""
);
CodeMirror.modeInfo.push({
name: "Dask SQL",
mime: "text/x-dasksql",
mode: "sql"
});
"""
)
display.display_javascript(js + _JS_ENABLE_DASK_SQL, raw=True)
def _create_set(keys: list[str]) -> dict[str, bool]: # pragma: no cover
"""Small helper function to turn a list into the correct format for codemirror"""
return {key: True for key in keys}
================================================
FILE: dask_sql/mappings.py
================================================
import logging
from datetime import datetime
from typing import Any
import dask.array as da
import dask.config as dask_config
import dask.dataframe as dd
import numpy as np
import pandas as pd
from dask_sql._datafusion_lib import DaskTypeMap, SqlTypeName
logger = logging.getLogger(__name__)
# Default mapping between python types and SQL types
_PYTHON_TO_SQL = {
np.float64: SqlTypeName.DOUBLE,
pd.Float64Dtype(): SqlTypeName.DOUBLE,
float: SqlTypeName.FLOAT,
np.float32: SqlTypeName.FLOAT,
pd.Float32Dtype(): SqlTypeName.FLOAT,
np.int64: SqlTypeName.BIGINT,
pd.Int64Dtype(): SqlTypeName.BIGINT,
int: SqlTypeName.INTEGER,
np.int32: SqlTypeName.INTEGER,
pd.Int32Dtype(): SqlTypeName.INTEGER,
np.int16: SqlTypeName.SMALLINT,
pd.Int16Dtype(): SqlTypeName.SMALLINT,
np.int8: SqlTypeName.TINYINT,
pd.Int8Dtype(): SqlTypeName.TINYINT,
np.uint64: SqlTypeName.BIGINT,
pd.UInt64Dtype(): SqlTypeName.BIGINT,
np.uint32: SqlTypeName.INTEGER,
pd.UInt32Dtype(): SqlTypeName.INTEGER,
np.uint16: SqlTypeName.SMALLINT,
pd.UInt16Dtype(): SqlTypeName.SMALLINT,
np.uint8: SqlTypeName.TINYINT,
pd.UInt8Dtype(): SqlTypeName.TINYINT,
np.bool_: SqlTypeName.BOOLEAN,
pd.BooleanDtype(): SqlTypeName.BOOLEAN,
str: SqlTypeName.VARCHAR,
np.object_: SqlTypeName.VARCHAR,
pd.StringDtype(): SqlTypeName.VARCHAR,
np.datetime64: SqlTypeName.TIMESTAMP,
}
# Default mapping between SQL types and python types
# for values
_SQL_TO_PYTHON_SCALARS = {
"SqlTypeName.DOUBLE": np.float64,
"SqlTypeName.FLOAT": np.float32,
"SqlTypeName.DECIMAL": np.float32,
"SqlTypeName.BIGINT": np.int64,
"SqlTypeName.INTEGER": np.int32,
"SqlTypeName.SMALLINT": np.int16,
"SqlTypeName.TINYINT": np.int8,
"SqlTypeName.BOOLEAN": np.bool_,
"SqlTypeName.VARCHAR": str,
"SqlTypeName.CHAR": str,
"SqlTypeName.NULL": type(None),
"SqlTypeName.SYMBOL": lambda x: x, # SYMBOL is a special type used for e.g. flags etc. We just keep it
}
# Default mapping between SQL types and python types
# for data frames
_SQL_TO_PYTHON_FRAMES = {
"SqlTypeName.DOUBLE": np.float64,
"SqlTypeName.FLOAT": np.float32,
"SqlTypeName.DECIMAL": np.float64, # We use np.float64 always, even though we might be able to use a smaller type
"SqlTypeName.BIGINT": pd.Int64Dtype(),
"SqlTypeName.INTEGER": pd.Int32Dtype(),
"SqlTypeName.SMALLINT": pd.Int16Dtype(),
"SqlTypeName.TINYINT": pd.Int8Dtype(),
"SqlTypeName.BOOLEAN": pd.BooleanDtype(),
"SqlTypeName.VARCHAR": pd.StringDtype(),
"SqlTypeName.CHAR": pd.StringDtype(),
"SqlTypeName.DATE": np.dtype(
"<M8[ns]"
), # TODO: ideally this would be np.dtype("<M8[D]") but that doesn't work for Pandas
"SqlTypeName.TIME": np.dtype("<M8[ns]"),
"SqlTypeName.TIMESTAMP": np.dtype("<M8[ns]"),
"SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE": pd.DatetimeTZDtype(
unit="ns", tz="UTC"
), # Everything is converted to UTC. So far, this did not break
"SqlTypeName.INTERVAL_DAY": np.dtype("<m8[ns]"),
"SqlTypeName.INTERVAL_MONTH_DAY_NANOSECOND": np.dtype("<m8[ns]"),
"SqlTypeName.NULL": type(None),
}
def python_to_sql_type(python_type) -> "DaskTypeMap":
"""Mapping between python and SQL types."""
if python_type in (int, float):
python_type = np.dtype(python_type)
elif python_type is str:
python_type = np.dtype("object")
if isinstance(python_type, np.dtype):
python_type = python_type.type
if isinstance(python_type, pd.DatetimeTZDtype):
return DaskTypeMap(
SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE,
unit=str(python_type.unit),
tz=str(python_type.tz),
)
if is_decimal(python_type):
return DaskTypeMap(
SqlTypeName.DECIMAL,
precision=python_type.precision,
scale=python_type.scale,
)
try:
return DaskTypeMap(_PYTHON_TO_SQL[python_type])
except KeyError: # pragma: no cover
raise NotImplementedError(
f"The python type {python_type} is not implemented (yet)"
)
def parse_datetime(obj):
formats = [
"%Y-%m-%d %H:%M:%S",
"%Y-%m-%d",
"%d-%m-%Y %H:%M:%S",
"%d-%m-%Y",
"%m/%d/%Y %H:%M:%S",
"%m/%d/%Y",
]
for f in formats:
try:
datetime_obj = datetime.strptime(obj, f)
return datetime_obj
except ValueError:
pass
raise ValueError("Unable to parse datetime: " + obj)
def sql_to_python_value(sql_type: "SqlTypeName", literal_value: Any) -> Any:
"""Mapping between SQL and python values (of correct type)."""
# In most of the cases, we turn the value first into a string.
# That might not be the most efficient thing to do,
# but works for all types (so far)
# Additionally, a literal type is not used
# so often anyways.
logger.debug(
f"sql_to_python_value -> sql_type: {sql_type} literal_value: {literal_value}"
)
if sql_type == SqlTypeName.CHAR or sql_type == SqlTypeName.VARCHAR:
# Some varchars contain an additional encoding
# in the format _ENCODING'string'
literal_value = str(literal_value)
if literal_value.startswith("_"):
encoding, literal_value = literal_value.split("'", 1)
literal_value = literal_value.rstrip("'")
literal_value = literal_value.encode(encoding=encoding)
return literal_value.decode(encoding=encoding)
return literal_value
elif (
sql_type == SqlTypeName.DECIMAL
and dask_config.get("sql.mappings.decimal_support") == "cudf"
):
from decimal import Decimal
python_type = Decimal
elif sql_type == SqlTypeName.INTERVAL_DAY:
return np.timedelta64(literal_value[0], "D") + np.timedelta64(
literal_value[1], "ms"
)
elif sql_type == SqlTypeName.INTERVAL:
# check for finer granular interval types, e.g., INTERVAL MONTH, INTERVAL YEAR
try:
interval_type = str(sql_type).split()[1].lower()
if interval_type in {"year", "quarter", "month"}:
# if sql_type is INTERVAL YEAR, Calcite will covert to months
delta = pd.tseries.offsets.DateOffset(months=float(str(literal_value)))
return delta
except IndexError: # pragma: no cover
# no finer granular interval type specified
pass
except TypeError: # pragma: no cover
# interval type is not recognized, fall back to default case
pass
# Calcite will always convert INTERVAL types except YEAR, QUATER, MONTH to milliseconds
# Issue: if sql_type is INTERVAL MICROSECOND, and value <= 1000, literal_value will be rounded to 0
return np.timedelta64(literal_value, "ms")
elif sql_type == SqlTypeName.INTERVAL_MONTH_DAY_NANOSECOND:
# DataFusion assumes 30 days per month. Therefore we multiply number of months by 30 and add to days
return np.timedelta64(
(literal_value[0] * 30) + literal_value[1], "D"
) + np.timedelta64(literal_value[2], "ns")
elif sql_type == SqlTypeName.BOOLEAN:
return bool(literal_value)
elif (
sql_type == SqlTypeName.TIMESTAMP
or sql_type == SqlTypeName.TIME
or sql_type == SqlTypeName.DATE
):
if isinstance(literal_value, str):
literal_value = parse_datetime(literal_value)
literal_value = np.datetime64(literal_value)
elif str(literal_value) == "None":
# NULL time
return pd.NaT # pragma: no cover
if sql_type == SqlTypeName.DATE:
return literal_value.astype("<M8[D]")
return literal_value.astype("<M8[ns]")
else:
try:
python_type = _SQL_TO_PYTHON_SCALARS[str(sql_type)]
except KeyError: # pragma: no cover
raise NotImplementedError(
f"The SQL type {sql_type} is not implemented (yet)"
)
literal_value = str(literal_value)
# empty literal type. We return NaN if possible
if literal_value == "None":
if isinstance(python_type(), np.floating):
return np.NaN
else:
return pd.NA
return python_type(literal_value)
def sql_to_python_type(sql_type: "SqlTypeName", *args) -> type:
"""Turn an SQL type into a dataframe dtype"""
try:
if (
sql_type == SqlTypeName.DECIMAL
and dask_config.get("sql.mappings.decimal_support") == "cudf"
):
try:
import cudf
except ImportError:
raise ModuleNotFoundError(
"Setting `sql.mappings.decimal_support=cudf` requires cudf"
)
return cudf.Decimal128Dtype(*args)
return _SQL_TO_PYTHON_FRAMES[str(sql_type)]
except KeyError: # pragma: no cover
raise NotImplementedError(
f"The SQL type {str(sql_type)} is not implemented (yet)"
)
def similar_type(lhs: type, rhs: type) -> bool:
"""
Measure simularity between types.
Two types are similar, if they both come from the same family,
e.g. both are ints, uints, floats, strings etc.
Size or precision is not taken into account.
TODO: nullability is not checked so far.
"""
pdt = pd.api.types
is_uint = pdt.is_unsigned_integer_dtype
is_sint = pdt.is_signed_integer_dtype
is_float = pdt.is_float_dtype
is_object = pdt.is_object_dtype
is_string = pdt.is_string_dtype
is_dt_ns = pdt.is_datetime64_ns_dtype
is_dt_tz = lambda t: is_dt_ns(t) and isinstance(t, pd.DatetimeTZDtype)
is_dt_ntz = lambda t: is_dt_ns(t) and not isinstance(t, pd.DatetimeTZDtype)
is_td_ns = pdt.is_timedelta64_ns_dtype
is_bool = pdt.is_bool_dtype
checks = [
is_uint,
is_sint,
is_float,
is_object,
# is_string_dtype considers decimal columns to be string columns
lambda x: is_string(x) and not is_decimal(x),
is_dt_tz,
is_dt_ntz,
is_td_ns,
is_bool,
is_decimal,
]
for check in checks:
if check(lhs) and check(rhs):
# check that decimal columns have equal precision/scale
if check is is_decimal:
return lhs.precision == rhs.precision and lhs.scale == rhs.scale
return True
return False
def cast_column_type(
df: dd.DataFrame, column_name: str, expected_type: type
) -> dd.DataFrame:
"""
Cast the type of the given column to the expected type,
if they are far "enough" away.
This means, a float will never be converted into a double
or a tinyint into another int - but a string to an integer etc.
"""
current_type = df[column_name].dtype
logger.debug(
f"Column {column_name} has type {current_type}, expecting {expected_type}..."
)
casted_column = cast_column_to_type(df[column_name], expected_type)
if casted_column is not None:
df[column_name] = casted_column
return df
def cast_column_to_type(col: dd.Series, expected_type: str):
"""Cast the given column to the expected type"""
pdt = pd.api.types
is_dt_ns = pdt.is_datetime64_ns_dtype
is_dt_tz = lambda t: is_dt_ns(t) and isinstance(t, pd.DatetimeTZDtype)
is_dt_ntz = lambda t: is_dt_ns(t) and not isinstance(t, pd.DatetimeTZDtype)
current_type = col.dtype
if similar_type(current_type, expected_type):
logger.debug("...not converting.")
return None
if pdt.is_integer_dtype(expected_type):
if pd.api.types.is_float_dtype(current_type):
logger.debug("...truncating...")
# Currently "trunc" can not be applied to NA (the pandas missing value type),
# because NA is a different type. It works with np.NaN though.
# For our use case, that does not matter, as the conversion to integer later
# will convert both NA and np.NaN to NA.
col = da.trunc(col.fillna(value=np.NaN))
elif pdt.is_timedelta64_dtype(current_type):
logger.debug(f"Explicitly casting from {current_type} to np.int64")
return col.astype(np.int64)
if is_dt_tz(current_type) and is_dt_ntz(expected_type):
# casting from timezone-aware to timezone-naive datatypes with astype is deprecated in pandas 2
return col.dt.tz_localize(None)
logger.debug(f"Need to cast from {current_type} to {expected_type}")
return col.astype(expected_type)
def is_decimal(dtype):
"""
Check if dtype is a decimal type
"""
return "decimal" in str(dtype).lower()
================================================
FILE: dask_sql/physical/__init__.py
================================================
================================================
FILE: dask_sql/physical/rel/__init__.py
================================================
from .convert import RelConverter
================================================
FILE: dask_sql/physical/rel/base.py
================================================
import logging
from typing import TYPE_CHECKING, Optional
import dask.dataframe as dd
from dask_sql.datacontainer import ColumnContainer, DataContainer
from dask_sql.mappings import cast_column_type, sql_to_python_type
if TYPE_CHECKING:
import dask_sql
from dask_sql._datafusion_lib import LogicalPlan, RelDataType
logger = logging.getLogger(__name__)
class BaseRelPlugin:
"""
Base class for all plugins to convert between
a RelNode to a python expression (dask dataframe).
Derived classed needs to override the class_name attribute
and the convert method.
"""
class_name = None
def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> dd.DataFrame:
"""Base method to implement"""
raise NotImplementedError
@staticmethod
def fix_column_to_row_type(
cc: ColumnContainer, row_type: "RelDataType", join_type: Optional[str] = None
) -> ColumnContainer:
"""
Make sure that the given column container
has the column names specified by the row type.
We assume that the column order is already correct
and will just "blindly" rename the columns.
"""
field_names = [str(x) for x in row_type.getFieldNames()]
if join_type in ("leftsemi", "leftanti"):
field_names = field_names[: len(cc.columns)]
logger.debug(f"Renaming {cc.columns} to {field_names}")
cc = cc.rename_handle_duplicates(
from_columns=cc.columns, to_columns=field_names
)
# TODO: We can also check for the types here and do any conversions if needed
return cc.limit_to(field_names)
@staticmethod
def check_columns_from_row_type(df: dd.DataFrame, row_type: "RelDataType"):
"""
Similar to `self.fix_column_to_row_type`, but this time
check for the correct column names instead of
applying them.
"""
field_names = [str(x) for x in row_type.getFieldNames()]
assert list(df.columns) == field_names
# TODO: similar to self.fix_column_to_row_type, we should check for the types
@staticmethod
def assert_inputs(
rel: "LogicalPlan",
n: int = 1,
context: "dask_sql.Context" = None,
) -> list[dd.DataFrame]:
"""
LogicalPlan nodes build on top of others.
Those are called the "input" of the LogicalPlan.
This function asserts that the given LogicalPlan has exactly as many
input tables as expected and returns them already
converted into a dask dataframe.
"""
input_rels = rel.get_inputs()
assert len(input_rels) == n
# Late import to remove cycling dependency
from dask_sql.physical.rel.convert import RelConverter
return [RelConverter.convert(input_rel, context) for input_rel in input_rels]
@staticmethod
def fix_dtype_to_row_type(
dc: DataContainer, row_type: "RelDataType", join_type: Optional[str] = None
):
"""
Fix the dtype of the given data container (or: the df within it)
to the data type given as argument.
To prevent unneeded conversions, do only convert if really needed,
e.g. if the two types are "similar" enough, do not convert.
Similarity involves the same general type (int, float, string etc)
but not necessary the size (int64 and int32 are compatible)
or the nullability.
TODO: we should check the nullability of the SQL type
"""
df = dc.df
cc = dc.column_container
field_list = row_type.getFieldList()
if join_type in ("leftsemi", "leftanti"):
field_list = field_list[: len(cc.columns)]
field_types = {
str(field.getQualifiedName()): field.getType() for field in field_list
}
for field_name, field_type in field_types.items():
sql_type = field_type.getSqlType()
sql_type_args = tuple()
if str(sql_type) == "SqlTypeName.DECIMAL":
sql_type_args = field_type.getDataType().getPrecisionScale()
expected_type = sql_to_python_type(sql_type, *sql_type_args)
df_field_name = cc.get_backend_by_frontend_name(field_name)
df = cast_column_type(df, df_field_name, expected_type)
return DataContainer(df, dc.column_container)
================================================
FILE: dask_sql/physical/rel/convert.py
================================================
import logging
from typing import TYPE_CHECKING
import dask.dataframe as dd
from dask_sql.physical.rel.base import BaseRelPlugin
from dask_sql.utils import LoggableDataFrame, Pluggable
if TYPE_CHECKING:
import dask_sql
from dask_sql._datafusion_lib import LogicalPlan
logger = logging.getLogger(__name__)
class RelConverter(Pluggable):
"""
Helper to convert from rel to a python expression
This class stores plugins which can convert from RelNodes to
python expression (typically dask dataframes).
The stored plugins are assumed to have a class attribute "class_name"
to control, which java classes they can convert
and they are expected to have a convert (instance) method
in the form
def convert(self, rel, context)
to do the actual conversion.
"""
@classmethod
def add_plugin_class(cls, plugin_class: BaseRelPlugin, replace=True):
"""Convenience function to add a class directly to the plugins"""
logger.debug(f"Registering REL plugin for {plugin_class.class_name}")
cls.add_plugin(plugin_class.class_name, plugin_class(), replace=replace)
@classmethod
def convert(cls, rel: "LogicalPlan", context: "dask_sql.Context") -> dd.DataFrame:
"""
Convert SQL AST tree node(s)
into a python expression (a dask dataframe)
using the stored plugins and the dictionary of
registered dask tables from the context.
The SQL AST tree is traversed. The context of the traversal is saved
in the Rust logic. We need to take that current node and determine
what "type" of Relational operator it represents to build the execution chain.
"""
node_type = rel.get_current_node_type()
try:
plugin_instance = cls.get_plugin(node_type)
except KeyError: # pragma: no cover
raise NotImplementedError(
f"No relational conversion for node type {node_type} available (yet)."
)
logger.debug(
f"Processing REL {rel} using {plugin_instance.__class__.__name__}..."
)
df = plugin_instance.convert(rel, context=context)
logger.debug(f"Processed REL {rel} into {LoggableDataFrame(df)}")
return df
================================================
FILE: dask_sql/physical/rel/custom/__init__.py
================================================
from .alter import AlterSchemaPlugin, AlterTablePlugin
from .analyze_table import AnalyzeTablePlugin
from .create_catalog_schema import CreateCatalogSchemaPlugin
from .create_experiment import CreateExperimentPlugin
from .create_memory_table import CreateMemoryTablePlugin
from .create_model import CreateModelPlugin
from .create_table import CreateTablePlugin
from .describe_model import DescribeModelPlugin
from .distributeby import DistributeByPlugin
from .drop_model import DropModelPlugin
from .drop_schema import DropSchemaPlugin
from .drop_table import DropTablePlugin
from .export_model import ExportModelPlugin
from .predict_model import PredictModelPlugin
from .show_columns import ShowColumnsPlugin
from .show_models import ShowModelsPlugin
from .show_schemas import ShowSchemasPlugin
from .show_tables import ShowTablesPlugin
from .use_schema import UseSchemaPlugin
__all__ = [
AnalyzeTablePlugin,
CreateExperimentPlugin,
CreateModelPlugin,
CreateCatalogSchemaPlugin,
CreateMemoryTablePlugin,
CreateTablePlugin,
DropModelPlugin,
DropSchemaPlugin,
DropTablePlugin,
ExportModelPlugin,
PredictModelPlugin,
ShowColumnsPlugin,
DescribeModelPlugin,
ShowModelsPlugin,
ShowSchemasPlugin,
ShowTablesPlugin,
UseSchemaPlugin,
AlterSchemaPlugin,
AlterTablePlugin,
DistributeByPlugin,
]
================================================
FILE: dask_sql/physical/rel/custom/alter.py
================================================
import logging
from typing import TYPE_CHECKING
from dask_sql.physical.rel.base import BaseRelPlugin
logger = logging.getLogger(__name__)
if TYPE_CHECKING:
import dask_sql
from dask_sql._datafusion_lib import LogicalPlan
class AlterSchemaPlugin(BaseRelPlugin):
"""
Alter schema name with new name;
ALTER SCHEMA <old-schema-name> RENAME TO <new-schema-name>
Using this SQL is equivalent to just doing
context.alter_schema(<old-schema-name>,<new-schema-name>)
but can also be used without writing a single line of code.
Nothing is returned.
"""
class_name = "AlterSchema"
def convert(self, rel: "LogicalPlan", context: "dask_sql.Context"):
alter_schema = rel.alter_schema()
old_schema_name = alter_schema.getOldSchemaName()
new_schema_name = alter_schema.getNewSchemaName()
logger.info(
f"changing schema name from `{old_schema_name}` to `{new_schema_name}`"
)
if old_schema_name not in context.schema:
raise KeyError(
f"Schema {old_schema_name} was not found, available schemas are - {context.schema.keys()}"
)
context.alter_schema(
old_schema_name=old_schema_name, new_schema_name=new_schema_name
)
class AlterTablePlugin(BaseRelPlugin):
"""
Alter table name with new name;
ALTER TABLE [IF EXISTS] <old-table-name> RENAME TO <new-table-name>
Using this SQL is equivalent to just doing
context.alter_table(<old-table-name>,<new-table-name>)
but can also be used without writing a single line of code.
Nothing is returned.
"""
class_name = "AlterTable"
def convert(self, rel: "LogicalPlan", context: "dask_sql.Context"):
alter_table = rel.alter_table()
old_table_name = alter_table.getOldTableName()
new_table_name = alter_table.getNewTableName()
schema_name = alter_table.getSchemaName() or context.schema_name
logger.info(
f"changing table name from `{old_table_name}` to `{new_table_name}`"
)
if old_table_name not in context.schema[schema_name].tables:
if not alter_table.getIfExists():
raise KeyError(
f"Table {old_table_name} was not found, available tables in {schema_name} are "
f"- {context.schema[schema_name].tables.keys()}"
)
else:
return
context.alter_table(
old_table_name=old_table_name,
new_table_name=new_table_name,
schema_name=schema_name,
)
================================================
FILE: dask_sql/physical/rel/custom/analyze_table.py
================================================
from typing import TYPE_CHECKING
import dask.dataframe as dd
import pandas as pd
from dask_sql.datacontainer import ColumnContainer, DataContainer
from dask_sql.mappings import python_to_sql_type
from dask_sql.physical.rel.base import BaseRelPlugin
if TYPE_CHECKING:
import dask_sql
from dask_sql._datafusion_lib import LogicalPlan
class AnalyzeTablePlugin(BaseRelPlugin):
"""
Show information on the table (like mean, max etc.)
on all or a subset of the columns..
The SQL is:
ANALYZE TABLE <table> COMPUTE STATISTICS FOR [ALL COLUMNS | COLUMNS a, b, ...]
The result is also a table, although it is created on the fly.
Please note: even though the syntax is very similar to e.g.
[the spark version](https://spark.apache.org/docs/3.0.0/sql-ref-syntax-aux-analyze-table.html),
this call does not help with query optimization (as the spark call would do),
as this is currently not implemented in dask-sql.
"""
class_name = "AnalyzeTable"
def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> DataContainer:
analyze_table = rel.analyze_table()
schema_name = analyze_table.getSchemaName() or context.schema_name
table_name = analyze_table.getTableName()
dc = context.schema[schema_name].tables[table_name]
columns = analyze_table.getColumns()
if not columns:
columns = dc.column_container.columns
# Define some useful shortcuts
mapping = dc.column_container.get_backend_by_frontend_name
df = dc.df
# Calculate statistics
statistics = dd.concat(
[
df[[mapping(col) for col in columns]].describe(),
pd.DataFrame(
{
mapping(col): str(
python_to_sql_type(df[mapping(col)].dtype)
).lower()
for col in columns
},
index=["data_type"],
),
pd.DataFrame(
{mapping(col): col for col in columns}, index=["col_name"]
),
]
)
cc = ColumnContainer(statistics.columns)
dc = DataContainer(statistics, cc)
return dc
================================================
FILE: dask_sql/physical/rel/custom/create_catalog_schema.py
================================================
import logging
from typing import TYPE_CHECKING
from dask_sql.physical.rel.base import BaseRelPlugin
if TYPE_CHECKING:
import dask_sql
from dask_sql._datafusion_lib import LogicalPlan
logger = logging.getLogger(__name__)
class CreateCatalogSchemaPlugin(BaseRelPlugin):
"""
Create a schema with the given name
and register it at the context.
The SQL call looks like
CREATE SCHEMA <schema-name>
Using this SQL is equivalent to just doing
context.create_schema(<schema-name>)
but can also be used without writing a single line of code.
Nothing is returned.
"""
class_name = "CreateCatalogSchema"
def convert(self, rel: "LogicalPlan", context: "dask_sql.Context"):
create_schema = rel.create_catalog_schema()
schema_name = create_schema.getSchemaName()
if schema_name in context.schema:
if create_schema.getIfNotExists():
return
elif not create_schema.getReplace():
raise RuntimeError(
f"A Schema with the name {schema_name} is already present."
)
context.create_schema(schema_name)
================================================
FILE: dask_sql/physical/rel/custom/create_experiment.py
================================================
import logging
from typing import TYPE_CHECKING
import dask.dataframe as dd
import pandas as pd
from dask_sql.datacontainer import ColumnContainer, DataContainer
from dask_sql.physical.rel.base import BaseRelPlugin
from dask_sql.physical.utils.ml_classes import get_cpu_classes, get_gpu_classes
from dask_sql.utils import convert_sql_kwargs, import_class, is_cudf_type
if TYPE_CHECKING:
import dask_sql
from dask_sql.rust import LogicalPlan
logger = logging.getLogger(__name__)
cpu_classes = get_cpu_classes()
gpu_classes = get_gpu_classes()
class CreateExperimentPlugin(BaseRelPlugin):
"""
Creates an Experiment for hyperparameter tuning or automl like behaviour,
i.e evaluates models with different hyperparameters and registers the best performing
model in the context with the name same as experiment name,
which can be used for prediction
sql syntax:
CREATE EXPERIMENT <name> WITH ( key = value )
AS <some select query>
OPTIONS:
* model_class: Class name or full path to the class of the model to train.
Any sklearn, cuML, XGBoost, or LightGBM classes can be inferred
without the full path. In this case, models trained on cuDF dataframes
are automatically mapped to cuML classes, and sklearn models otherwise.
We map to cuML-Dask based models when possible and single-GPU cuML models otherwise.
Any model class with sklearn interface is valid, but might or
might not work well with Dask dataframes.
You might need to install necessary packages to use
the models.
* experiment_class : Class name or full path of the Hyperparameter tuner.
Any sklearn or cuML classes can be inferred
without the full path. In this case, models trained on cuDF dataframes
are automatically mapped to cuML classes, and sklearn models otherwise.
* tune_parameters:
Key-value of pairs of Hyperparameters to tune, i.e Search Space for
particular model to tune
* automl_class : Full path of the class which is sklearn compatible and
able to distribute work to dask clusters, currently tested with
tpot automl framework.
Refer : [Tpot example](https://examples.dask.org/machine-learning/tpot.html)
* target_column: Which column from the data to use as target.
Currently this parameter is required field, because tuning and automl
behaviour is implemented only for supervised algorithms.
* automl_kwargs:
Key-value pairs of arguments to be passed to automl class .
Refer : [Using Tpot parameters](https://epistasislab.github.io/tpot/using/)
* experiment_kwargs:
Use this parameter for passing any keyword arguments to experiment class
* tune_fit_kwargs:
Use this parameter for passing any keyword arguments to experiment.fit() method
example:
for Hyperparameter tuning : (Train and evaluate same model with different parameters)
CREATE EXPERIMENT my_exp WITH(
model_class = 'sklearn.ensemble.GradientBoostingClassifier',
experiment_class = 'sklearn.model_selection.GridSearchCV',
tune_parameters = (n_estimators = ARRAY [16, 32, 2],
learning_rate = ARRAY [0.1,0.01,0.001],
max_depth = ARRAY [3,4,5,10]
),
target_column = 'target'
) AS (
SELECT x, y, x*y > 0 AS target
FROM timeseries
LIMIT 100
)
for automl : (Train different different model with different parameter)
CREATE EXPERIMENT my_exp WITH (
automl_class = 'tpot.TPOTClassifier',
automl_kwargs = (population_size = 2 ,
generations=2,
cv=2,
n_jobs=-1,
use_dask=True,
max_eval_time_mins=1),
target_column = 'target'
) AS (
SELECT x, y, x*y > 0 AS target
FROM timeseries
LIMIT 100
)
"""
class_name = "CreateExperiment"
def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> DataContainer:
create_experiment = rel.create_experiment()
select = create_experiment.getSelectQuery()
schema_name = create_experiment.getSchemaName() or context.schema_name
experiment_name = create_experiment.getExperimentName()
kwargs = convert_sql_kwargs(create_experiment.getSQLWithOptions())
if experiment_name in context.schema[schema_name].experiments:
if create_experiment.getIfNotExists():
return
elif not create_experiment.getOrReplace():
raise RuntimeError(
f"A experiment with the name {experiment_name} is already present."
)
logger.debug(
f"Creating Experiment {experiment_name} from query {select} with options {kwargs}"
)
model_class = None
automl_class = None
experiment_class = None
if "model_class" in kwargs:
model_class = kwargs.pop("model_class")
# when model class was provided, must provide experiment_class also for tuning
if "experiment_class" not in kwargs:
raise ValueError(
f"Parameters must include a 'experiment_class' parameter for tuning {model_class}."
)
experiment_class = kwargs.pop("experiment_class")
elif "automl_class" in kwargs:
automl_class = kwargs.pop("automl_class")
else:
raise ValueError(
"Parameters must include a 'model_class' or 'automl_class' parameter."
)
target_column = kwargs.pop("target_column", "")
tune_fit_kwargs = kwargs.pop("tune_fit_kwargs", {})
parameters = kwargs.pop("tune_parameters", {})
experiment_kwargs = kwargs.pop("experiment_kwargs", {})
automl_kwargs = kwargs.pop("automl_kwargs", {})
logger.info(parameters)
training_df = context.sql(select)
if not target_column:
raise ValueError(
"Unsupervised Algorithm cannot be tuned Automatically,"
"Consider providing 'target column'"
)
non_target_columns = [
col for col in training_df.columns if col != target_column
]
X = training_df[non_target_columns]
y = training_df[target_column]
if model_class and experiment_class:
if is_cudf_type(training_df):
model_class = gpu_classes.get(model_class, model_class)
experiment_class = gpu_classes.get(experiment_class, experiment_class)
else:
model_class = cpu_classes.get(model_class, model
gitextract_q18bbzy0/
├── .cargo/
│ └── config.toml
├── .coveragerc
├── .dockerignore
├── .github/
│ ├── CODEOWNERS
│ ├── ISSUE_TEMPLATE/
│ │ ├── bug_report.md
│ │ ├── documentation-request.md
│ │ ├── feature_request.md
│ │ └── submit-question.md
│ ├── dependabot.yml
│ └── workflows/
│ ├── conda.yml
│ ├── docker.yml
│ ├── release.yml
│ ├── rust.yml
│ ├── style.yml
│ ├── test-upstream.yml
│ └── test.yml
├── .gitignore
├── .pre-commit-config.yaml
├── .readthedocs.yaml
├── CODE_OF_CONDUCT.md
├── CONTRIBUTING.md
├── Cargo.toml
├── LICENSE.txt
├── MANIFEST.in
├── README.md
├── conftest.py
├── continuous_integration/
│ ├── docker/
│ │ ├── cloud.dockerfile
│ │ ├── conda.txt
│ │ └── main.dockerfile
│ ├── environment-3.10.yaml
│ ├── environment-3.11.yaml
│ ├── environment-3.12.yaml
│ ├── environment-3.9.yaml
│ ├── gpuci/
│ │ ├── environment-3.10.yaml
│ │ ├── environment-3.11.yaml
│ │ └── environment-3.9.yaml
│ ├── recipe/
│ │ ├── build.sh
│ │ ├── conda_build_config.yaml
│ │ ├── meta.yaml
│ │ └── run_test.py
│ └── scripts/
│ ├── startup_script.py
│ └── update-dependencies.sh
├── dask_sql/
│ ├── __init__.py
│ ├── _compat.py
│ ├── cmd.py
│ ├── config.py
│ ├── context.py
│ ├── datacontainer.py
│ ├── input_utils/
│ │ ├── __init__.py
│ │ ├── base.py
│ │ ├── convert.py
│ │ ├── dask.py
│ │ ├── hive.py
│ │ ├── intake.py
│ │ ├── location.py
│ │ ├── pandaslike.py
│ │ └── sqlalchemy.py
│ ├── integrations/
│ │ ├── __init__.py
│ │ ├── fugue.py
│ │ └── ipython.py
│ ├── mappings.py
│ ├── physical/
│ │ ├── __init__.py
│ │ ├── rel/
│ │ │ ├── __init__.py
│ │ │ ├── base.py
│ │ │ ├── convert.py
│ │ │ ├── custom/
│ │ │ │ ├── __init__.py
│ │ │ │ ├── alter.py
│ │ │ │ ├── analyze_table.py
│ │ │ │ ├── create_catalog_schema.py
│ │ │ │ ├── create_experiment.py
│ │ │ │ ├── create_memory_table.py
│ │ │ │ ├── create_model.py
│ │ │ │ ├── create_table.py
│ │ │ │ ├── describe_model.py
│ │ │ │ ├── distributeby.py
│ │ │ │ ├── drop_model.py
│ │ │ │ ├── drop_schema.py
│ │ │ │ ├── drop_table.py
│ │ │ │ ├── export_model.py
│ │ │ │ ├── metrics.py
│ │ │ │ ├── predict_model.py
│ │ │ │ ├── show_columns.py
│ │ │ │ ├── show_models.py
│ │ │ │ ├── show_schemas.py
│ │ │ │ ├── show_tables.py
│ │ │ │ ├── use_schema.py
│ │ │ │ └── wrappers.py
│ │ │ └── logical/
│ │ │ ├── __init__.py
│ │ │ ├── aggregate.py
│ │ │ ├── cross_join.py
│ │ │ ├── empty.py
│ │ │ ├── explain.py
│ │ │ ├── filter.py
│ │ │ ├── join.py
│ │ │ ├── limit.py
│ │ │ ├── project.py
│ │ │ ├── sample.py
│ │ │ ├── sort.py
│ │ │ ├── subquery_alias.py
│ │ │ ├── table_scan.py
│ │ │ ├── union.py
│ │ │ ├── values.py
│ │ │ └── window.py
│ │ ├── rex/
│ │ │ ├── __init__.py
│ │ │ ├── base.py
│ │ │ ├── convert.py
│ │ │ └── core/
│ │ │ ├── __init__.py
│ │ │ ├── alias.py
│ │ │ ├── call.py
│ │ │ ├── input_ref.py
│ │ │ ├── literal.py
│ │ │ └── subquery.py
│ │ └── utils/
│ │ ├── __init__.py
│ │ ├── filter.py
│ │ ├── groupby.py
│ │ ├── ml_classes.py
│ │ ├── sort.py
│ │ └── statistics.py
│ ├── server/
│ │ ├── __init__.py
│ │ ├── app.py
│ │ ├── presto_jdbc.py
│ │ └── responses.py
│ ├── sql-schema.yaml
│ ├── sql.yaml
│ └── utils.py
├── docs/
│ ├── Makefile
│ ├── environment.yml
│ ├── make.bat
│ ├── requirements-docs.txt
│ └── source/
│ ├── api.rst
│ ├── best_practices.rst
│ ├── cmd.rst
│ ├── conf.py
│ ├── configuration.rst
│ ├── custom.rst
│ ├── data_input.rst
│ ├── fugue.rst
│ ├── how_does_it_work.rst
│ ├── index.rst
│ ├── installation.rst
│ ├── machine_learning.rst
│ ├── quickstart.rst
│ ├── server.rst
│ ├── sql/
│ │ ├── creation.rst
│ │ ├── describe.rst
│ │ ├── ml.rst
│ │ └── select.rst
│ └── sql.rst
├── notebooks/
│ ├── Custom Functions.ipynb
│ ├── Feature Overview.ipynb
│ ├── FugueSQL.ipynb
│ └── iris.csv
├── pyproject.toml
├── rustfmt.toml
├── setup.cfg
├── src/
│ ├── dialect.rs
│ ├── error.rs
│ ├── expression.rs
│ ├── lib.rs
│ ├── parser.rs
│ ├── sql/
│ │ ├── column.rs
│ │ ├── exceptions.rs
│ │ ├── function.rs
│ │ ├── logical/
│ │ │ ├── aggregate.rs
│ │ │ ├── alter_schema.rs
│ │ │ ├── alter_table.rs
│ │ │ ├── analyze_table.rs
│ │ │ ├── create_catalog_schema.rs
│ │ │ ├── create_experiment.rs
│ │ │ ├── create_memory_table.rs
│ │ │ ├── create_model.rs
│ │ │ ├── create_table.rs
│ │ │ ├── describe_model.rs
│ │ │ ├── drop_model.rs
│ │ │ ├── drop_schema.rs
│ │ │ ├── drop_table.rs
│ │ │ ├── empty_relation.rs
│ │ │ ├── explain.rs
│ │ │ ├── export_model.rs
│ │ │ ├── filter.rs
│ │ │ ├── join.rs
│ │ │ ├── limit.rs
│ │ │ ├── predict_model.rs
│ │ │ ├── projection.rs
│ │ │ ├── repartition_by.rs
│ │ │ ├── show_columns.rs
│ │ │ ├── show_models.rs
│ │ │ ├── show_schemas.rs
│ │ │ ├── show_tables.rs
│ │ │ ├── sort.rs
│ │ │ ├── subquery_alias.rs
│ │ │ ├── table_scan.rs
│ │ │ ├── use_schema.rs
│ │ │ └── window.rs
│ │ ├── logical.rs
│ │ ├── optimizer/
│ │ │ ├── decorrelate_where_exists.rs
│ │ │ ├── decorrelate_where_in.rs
│ │ │ ├── dynamic_partition_pruning.rs
│ │ │ ├── join_reorder.rs
│ │ │ └── utils.rs
│ │ ├── optimizer.rs
│ │ ├── parser_utils.rs
│ │ ├── preoptimizer.rs
│ │ ├── schema.rs
│ │ ├── statement.rs
│ │ ├── table.rs
│ │ ├── types/
│ │ │ ├── rel_data_type.rs
│ │ │ └── rel_data_type_field.rs
│ │ └── types.rs
│ └── sql.rs
└── tests/
├── __init__.py
├── integration/
│ ├── __init__.py
│ ├── fixtures.py
│ ├── test_analyze.py
│ ├── test_cmd.py
│ ├── test_compatibility.py
│ ├── test_complex.py
│ ├── test_create.py
│ ├── test_distributeby.py
│ ├── test_explain.py
│ ├── test_filter.py
│ ├── test_fugue.py
│ ├── test_function.py
│ ├── test_groupby.py
│ ├── test_hive.py
│ ├── test_intake.py
│ ├── test_jdbc.py
│ ├── test_join.py
│ ├── test_model.py
│ ├── test_over.py
│ ├── test_postgres.py
│ ├── test_rex.py
│ ├── test_sample.py
│ ├── test_schema.py
│ ├── test_select.py
│ ├── test_server.py
│ ├── test_show.py
│ ├── test_sort.py
│ ├── test_sqlite.py
│ └── test_union.py
├── unit/
│ ├── __init__.py
│ ├── test_call.py
│ ├── test_config.py
│ ├── test_context.py
│ ├── test_datacontainer.py
│ ├── test_mapping.py
│ ├── test_ml_utils.py
│ ├── test_queries.py
│ ├── test_statistics.py
│ └── test_utils.py
└── utils.py
SYMBOL INDEX (1675 symbols across 162 files)
FILE: conftest.py
function pytest_addoption (line 7) | def pytest_addoption(parser):
function pytest_runtest_setup (line 14) | def pytest_runtest_setup(item):
function data_dir (line 28) | def data_dir(request):
function queries_dir (line 33) | def queries_dir(request):
FILE: dask_sql/cmd.py
class CompatiblePromptSession (line 33) | class CompatiblePromptSession:
method __init__ (line 46) | def __init__(self, lexer) -> None: # pragma: no cover
function _display_markdown (line 69) | def _display_markdown(content, **kwargs):
function _parse_meta_command (line 74) | def _parse_meta_command(sql):
function _meta_commands (line 79) | def _meta_commands(sql: str, context: Context, client: Client) -> Union[...
function cmd_loop (line 149) | def cmd_loop(
function main (line 233) | def main(): # pragma: no cover
FILE: dask_sql/context.py
class Context (line 51) | class Context:
method __init__ (line 85) | def __init__(self, logging_level=logging.INFO):
method create_table (line 168) | def create_table(
method drop_table (line 295) | def drop_table(self, table_name: str, schema_name: str = None):
method drop_schema (line 307) | def drop_schema(self, schema_name: str):
method register_function (line 324) | def register_function(
method register_aggregation (line 415) | def register_aggregation(
method sql (line 482) | def sql(
method explain (line 535) | def explain(
method visualize (line 573) | def visualize(self, sql: str, filename="mydask.png") -> None: # pragm...
method create_schema (line 580) | def create_schema(self, schema_name: str):
method alter_schema (line 589) | def alter_schema(self, old_schema_name, new_schema_name):
method alter_table (line 599) | def alter_table(self, old_table_name, new_table_name, schema_name=None):
method register_experiment (line 615) | def register_experiment(
method register_model (line 626) | def register_model(
method ipython_magic (line 651) | def ipython_magic(
method run_server (line 704) | def run_server(self, **kwargs): # pragma: no cover
method stop_server (line 721) | def stop_server(self): # pragma: no cover
method fqn (line 732) | def fqn(self, tbl: "DaskTable") -> tuple[str, str]:
method _prepare_schemas (line 749) | def _prepare_schemas(self):
method _get_ral (line 819) | def _get_ral(self, sql):
method _compute_table_from_rel (line 874) | def _compute_table_from_rel(self, rel: "LogicalPlan", return_futures: ...
method _get_tables_from_stack (line 914) | def _get_tables_from_stack(self):
method _register_callable (line 933) | def _register_callable(
FILE: dask_sql/datacontainer.py
class ColumnContainer (line 14) | class ColumnContainer:
method __init__ (line 29) | def __init__(
method _copy (line 45) | def _copy(self) -> ColumnContainer:
method limit_to (line 53) | def limit_to(self, fields: list[str]) -> ColumnContainer:
method rename (line 67) | def rename(self, columns: dict[str, str]) -> ColumnContainer:
method rename_handle_duplicates (line 86) | def rename_handle_duplicates(
method mapping (line 108) | def mapping(self) -> list[tuple[str, ColumnType]]:
method columns (line 115) | def columns(self) -> list[str]:
method add (line 121) | def add(
method get_backend_by_frontend_index (line 141) | def get_backend_by_frontend_index(self, index: int) -> str:
method get_backend_by_frontend_name (line 150) | def get_backend_by_frontend_name(self, column: str) -> str:
method make_unique (line 161) | def make_unique(self, prefix="col"):
class ColumnContainer (line 19) | class ColumnContainer:
method __init__ (line 29) | def __init__(
method _copy (line 45) | def _copy(self) -> ColumnContainer:
method limit_to (line 53) | def limit_to(self, fields: list[str]) -> ColumnContainer:
method rename (line 67) | def rename(self, columns: dict[str, str]) -> ColumnContainer:
method rename_handle_duplicates (line 86) | def rename_handle_duplicates(
method mapping (line 108) | def mapping(self) -> list[tuple[str, ColumnType]]:
method columns (line 115) | def columns(self) -> list[str]:
method add (line 121) | def add(
method get_backend_by_frontend_index (line 141) | def get_backend_by_frontend_index(self, index: int) -> str:
method get_backend_by_frontend_name (line 150) | def get_backend_by_frontend_name(self, column: str) -> str:
method make_unique (line 161) | def make_unique(self, prefix="col"):
class Statistics (line 174) | class Statistics:
method __init__ (line 181) | def __init__(self, row_count: int) -> None:
method __eq__ (line 184) | def __eq__(self, other):
class DataContainer (line 190) | class DataContainer:
method __init__ (line 205) | def __init__(
method assign (line 217) | def assign(self) -> dd.DataFrame:
class UDF (line 234) | class UDF:
method __init__ (line 235) | def __init__(self, func, row_udf: bool, params, return_type=None):
method __call__ (line 252) | def __call__(self, *args, **kwargs):
method __eq__ (line 272) | def __eq__(self, other):
method __hash__ (line 277) | def __hash__(self):
class SchemaContainer (line 281) | class SchemaContainer:
method __init__ (line 282) | def __init__(self, name: str):
FILE: dask_sql/input_utils/base.py
class BaseInputPlugin (line 4) | class BaseInputPlugin:
method is_correct_input (line 5) | def is_correct_input(
method to_dc (line 10) | def to_dc(self, input_item: Any, table_name: str, format: str = None, ...
FILE: dask_sql/input_utils/convert.py
class InputUtil (line 30) | class InputUtil(Pluggable):
method add_plugin_class (line 37) | def add_plugin_class(cls, plugin_class: BaseInputPlugin, replace=True):
method to_dc (line 43) | def to_dc(
method _get_dask_dataframe (line 76) | def _get_dask_dataframe(
FILE: dask_sql/input_utils/dask.py
class DaskInputPlugin (line 8) | class DaskInputPlugin(BaseInputPlugin):
method is_correct_input (line 11) | def is_correct_input(
method to_dc (line 16) | def to_dc(
FILE: dask_sql/input_utils/hive.py
class HiveInputPlugin (line 27) | class HiveInputPlugin(BaseInputPlugin):
method is_correct_input (line 30) | def is_correct_input(
method is_sqlalchemy_hive (line 37) | def is_sqlalchemy_hive(self, input_item: Any):
method to_dc (line 40) | def to_dc(
method _parse_hive_table_description (line 192) | def _parse_hive_table_description(
method _parse_hive_partition_description (line 281) | def _parse_hive_partition_description(
method _fetch_all_results (line 299) | def _fetch_all_results(
FILE: dask_sql/input_utils/intake.py
class IntakeCatalogInputPlugin (line 11) | class IntakeCatalogInputPlugin(BaseInputPlugin):
method is_correct_input (line 14) | def is_correct_input(
method to_dc (line 21) | def to_dc(
FILE: dask_sql/input_utils/location.py
class LocationInputPlugin (line 11) | class LocationInputPlugin(BaseInputPlugin):
method is_correct_input (line 14) | def is_correct_input(
method to_dc (line 19) | def to_dc(
FILE: dask_sql/input_utils/pandaslike.py
class PandasLikeInputPlugin (line 7) | class PandasLikeInputPlugin(BaseInputPlugin):
method is_correct_input (line 10) | def is_correct_input(
method to_dc (line 18) | def to_dc(
FILE: dask_sql/input_utils/sqlalchemy.py
class SqlalchemyHiveInputPlugin (line 6) | class SqlalchemyHiveInputPlugin(HiveInputPlugin):
method is_correct_input (line 9) | def is_correct_input(
method to_dc (line 18) | def to_dc(
FILE: dask_sql/integrations/fugue.py
function _register_engines (line 22) | def _register_engines() -> None:
class DaskSQLEngine (line 41) | class DaskSQLEngine(fugue.execution.execution_engine.SQLEngine):
method __init__ (line 51) | def __init__(self, *args, **kwargs):
method is_distributed (line 56) | def is_distributed(self) -> bool:
method select (line 59) | def select(
class DaskSQLExecutionEngine (line 72) | class DaskSQLExecutionEngine(fugue_dask.DaskExecutionEngine):
method __init__ (line 82) | def __init__(self, *args, **kwargs):
method default_sql_engine (line 88) | def default_sql_engine(self) -> fugue.execution.execution_engine.SQLEn...
function fsql_dask (line 92) | def fsql_dask(
FILE: dask_sql/integrations/ipython.py
function ipython_integration (line 54) | def ipython_integration(
function _register_ipython_magic (line 65) | def _register_ipython_magic(
function _register_syntax_highlighting (line 103) | def _register_syntax_highlighting(): # pragma: no cover
function _create_set (line 156) | def _create_set(keys: list[str]) -> dict[str, bool]: # pragma: no cover
FILE: dask_sql/mappings.py
function python_to_sql_type (line 92) | def python_to_sql_type(python_type) -> "DaskTypeMap":
function parse_datetime (line 125) | def parse_datetime(obj):
function sql_to_python_value (line 145) | def sql_to_python_value(sql_type: "SqlTypeName", literal_value: Any) -> ...
function sql_to_python_type (line 243) | def sql_to_python_type(sql_type: "SqlTypeName", *args) -> type:
function similar_type (line 264) | def similar_type(lhs: type, rhs: type) -> bool:
function cast_column_type (line 309) | def cast_column_type(
function cast_column_to_type (line 332) | def cast_column_to_type(col: dd.Series, expected_type: str):
function is_decimal (line 366) | def is_decimal(dtype):
FILE: dask_sql/physical/rel/base.py
class BaseRelPlugin (line 16) | class BaseRelPlugin:
method convert (line 27) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
method fix_column_to_row_type (line 32) | def fix_column_to_row_type(
method check_columns_from_row_type (line 54) | def check_columns_from_row_type(df: dd.DataFrame, row_type: "RelDataTy...
method assert_inputs (line 67) | def assert_inputs(
method fix_dtype_to_row_type (line 89) | def fix_dtype_to_row_type(
FILE: dask_sql/physical/rel/convert.py
class RelConverter (line 16) | class RelConverter(Pluggable):
method add_plugin_class (line 33) | def add_plugin_class(cls, plugin_class: BaseRelPlugin, replace=True):
method convert (line 39) | def convert(cls, rel: "LogicalPlan", context: "dask_sql.Context") -> d...
FILE: dask_sql/physical/rel/custom/alter.py
class AlterSchemaPlugin (line 13) | class AlterSchemaPlugin(BaseRelPlugin):
method convert (line 29) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context"):
class AlterTablePlugin (line 47) | class AlterTablePlugin(BaseRelPlugin):
method convert (line 63) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context"):
FILE: dask_sql/physical/rel/custom/analyze_table.py
class AnalyzeTablePlugin (line 15) | class AnalyzeTablePlugin(BaseRelPlugin):
method convert (line 33) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/custom/create_catalog_schema.py
class CreateCatalogSchemaPlugin (line 13) | class CreateCatalogSchemaPlugin(BaseRelPlugin):
method convert (line 31) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context"):
FILE: dask_sql/physical/rel/custom/create_experiment.py
class CreateExperimentPlugin (line 22) | class CreateExperimentPlugin(BaseRelPlugin):
method convert (line 103) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/custom/create_memory_table.py
class CreateMemoryTablePlugin (line 14) | class CreateMemoryTablePlugin(BaseRelPlugin):
method convert (line 37) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/custom/create_model.py
class CreateModelPlugin (line 23) | class CreateModelPlugin(BaseRelPlugin):
method convert (line 112) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/custom/create_table.py
class CreateTablePlugin (line 15) | class CreateTablePlugin(BaseRelPlugin):
method convert (line 41) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/custom/describe_model.py
class DescribeModelPlugin (line 14) | class DescribeModelPlugin(BaseRelPlugin):
method convert (line 27) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/custom/distributeby.py
class DistributeByPlugin (line 15) | class DistributeByPlugin(BaseRelPlugin):
method convert (line 26) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/custom/drop_model.py
class DropModelPlugin (line 14) | class DropModelPlugin(BaseRelPlugin):
method convert (line 24) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/custom/drop_schema.py
class DropSchemaPlugin (line 13) | class DropSchemaPlugin(BaseRelPlugin):
method convert (line 23) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context"):
FILE: dask_sql/physical/rel/custom/drop_table.py
class DropTablePlugin (line 14) | class DropTablePlugin(BaseRelPlugin):
method convert (line 24) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/custom/export_model.py
class ExportModelPlugin (line 15) | class ExportModelPlugin(BaseRelPlugin):
method convert (line 47) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context"):
FILE: dask_sql/physical/rel/custom/metrics.py
function accuracy_score (line 16) | def accuracy_score(
function _log_loss_inner (line 73) | def _log_loss_inner(
function log_loss (line 86) | def log_loss(
function _check_sample_weight (line 129) | def _check_sample_weight(sample_weight: Optional[ArrayLike]):
function mean_squared_error (line 135) | def mean_squared_error(
function _check_reg_targets (line 162) | def _check_reg_targets(
function r2_score (line 178) | def r2_score(
FILE: dask_sql/physical/rel/custom/predict_model.py
class PredictModelPlugin (line 18) | class PredictModelPlugin(BaseRelPlugin):
method convert (line 53) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/custom/show_columns.py
class ShowColumnsPlugin (line 15) | class ShowColumnsPlugin(BaseRelPlugin):
method convert (line 27) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/custom/show_models.py
class ShowModelsPlugin (line 14) | class ShowModelsPlugin(BaseRelPlugin):
method convert (line 26) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/custom/show_schemas.py
class ShowSchemasPlugin (line 14) | class ShowSchemasPlugin(BaseRelPlugin):
method convert (line 26) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/custom/show_tables.py
class ShowTablesPlugin (line 14) | class ShowTablesPlugin(BaseRelPlugin):
method convert (line 29) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/custom/use_schema.py
class UseSchemaPlugin (line 11) | class UseSchemaPlugin(BaseRelPlugin):
method convert (line 23) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/custom/wrappers.py
class ParallelPostFit (line 51) | class ParallelPostFit(sklearn.base.BaseEstimator, sklearn.base.MetaEstim...
method __init__ (line 101) | def __init__(
method _check_array (line 115) | def _check_array(self, X):
method _postfit_estimator (line 142) | def _postfit_estimator(self):
method fit (line 146) | def fit(self, X, y=None, **kwargs):
method partial_fit (line 167) | def partial_fit(self, X, y=None, **kwargs):
method transform (line 176) | def transform(self, X):
method score (line 233) | def score(self, X, y, compute=True):
method predict (line 275) | def predict(self, X):
method predict_proba (line 331) | def predict_proba(self, X):
method predict_log_proba (line 388) | def predict_log_proba(self, X):
method _check_method (line 409) | def _check_method(self, method):
class Incremental (line 425) | class Incremental(ParallelPostFit):
method __init__ (line 493) | def __init__(
method _postfit_estimator (line 516) | def _postfit_estimator(self):
method _fit_for_estimator (line 520) | def _fit_for_estimator(self, estimator, X, y, **fit_kwargs):
method fit (line 544) | def fit(self, X, y=None, **fit_kwargs):
method partial_fit (line 549) | def partial_fit(self, X, y=None, **fit_kwargs):
function handle_empty_partitions (line 569) | def handle_empty_partitions(output_meta):
function _predict (line 598) | def _predict(part, estimator, output_meta=None):
function _predict_proba (line 606) | def _predict_proba(part, estimator, output_meta=None):
function _transform (line 614) | def _transform(part, estimator, output_meta=None):
function _get_output_dask_ar_meta_for_estimator (line 622) | def _get_output_dask_ar_meta_for_estimator(model_fn, estimator, input_da...
function copy_learned_attributes (line 673) | def copy_learned_attributes(from_estimator, to_estimator):
function get_scorer (line 680) | def get_scorer(scoring: Union[str, Callable], compute: bool = True) -> C...
function check_scoring (line 710) | def check_scoring(estimator, scoring=None, **kwargs):
function fit (line 718) | def fit(
function _blocks_and_name (line 800) | def _blocks_and_name(obj):
function _partial_fit (line 818) | def _partial_fit(model, x, y, kwargs=None):
FILE: dask_sql/physical/rel/logical/aggregate.py
class ReduceAggregation (line 24) | class ReduceAggregation(dd.Aggregation):
method __init__ (line 30) | def __init__(self, name: str, operation: Callable):
class AggregationOnPandas (line 36) | class AggregationOnPandas(dd.Aggregation):
method __init__ (line 45) | def __init__(self, function_name: str):
class AggregationSpecification (line 52) | class AggregationSpecification:
method __init__ (line 63) | def __init__(self, built_in_aggregation, custom_aggregation=None):
method get_supported_aggregation (line 67) | def get_supported_aggregation(self, series):
class DaskAggregatePlugin (line 91) | class DaskAggregatePlugin(BaseRelPlugin):
method convert (line 233) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
method _do_aggregations (line 288) | def _do_aggregations(
method _collect_aggregations (line 377) | def _collect_aggregations(
method _perform_aggregation (line 522) | def _perform_aggregation(
FILE: dask_sql/physical/rel/logical/cross_join.py
class DaskCrossJoinPlugin (line 15) | class DaskCrossJoinPlugin(BaseRelPlugin):
method convert (line 25) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/logical/empty.py
class DaskEmptyRelationPlugin (line 17) | class DaskEmptyRelationPlugin(BaseRelPlugin):
method convert (line 25) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/logical/explain.py
class ExplainPlugin (line 10) | class ExplainPlugin(BaseRelPlugin):
method convert (line 17) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context"):
FILE: dask_sql/physical/rel/logical/filter.py
function filter_or_scalar (line 20) | def filter_or_scalar(
class DaskFilterPlugin (line 48) | class DaskFilterPlugin(BaseRelPlugin):
method convert (line 56) | def convert(
FILE: dask_sql/physical/rel/logical/join.py
class DaskJoinPlugin (line 23) | class DaskJoinPlugin(BaseRelPlugin):
method convert (line 50) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
method _join_on_columns (line 189) | def _join_on_columns(
method _split_join_condition (line 250) | def _split_join_condition(
method _extract_lhs_rhs (line 274) | def _extract_lhs_rhs(self, rex):
FILE: dask_sql/physical/rel/logical/limit.py
class DaskLimitPlugin (line 18) | class DaskLimitPlugin(BaseRelPlugin):
method convert (line 26) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
method _apply_limit (line 48) | def _apply_limit(self, df: dd.DataFrame, limit: int, offset: int) -> d...
FILE: dask_sql/physical/rel/logical/project.py
class DaskProjectPlugin (line 17) | class DaskProjectPlugin(BaseRelPlugin):
method convert (line 26) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/logical/sample.py
class SamplePlugin (line 16) | class SamplePlugin(BaseRelPlugin):
method convert (line 37) | def convert(
FILE: dask_sql/physical/rel/logical/sort.py
class DaskSortPlugin (line 12) | class DaskSortPlugin(BaseRelPlugin):
method convert (line 19) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/logical/subquery_alias.py
class SubqueryAlias (line 11) | class SubqueryAlias(BaseRelPlugin):
method convert (line 18) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context"):
FILE: dask_sql/physical/rel/logical/table_scan.py
class DaskTableScanPlugin (line 21) | class DaskTableScanPlugin(BaseRelPlugin):
method convert (line 34) | def convert(
method _apply_projections (line 61) | def _apply_projections(self, table_scan, dask_table, dc):
method _apply_filters (line 80) | def _apply_filters(self, table_scan, rel, dc, context):
FILE: dask_sql/physical/rel/logical/union.py
function _extract_df (line 13) | def _extract_df(obj_cc, obj_df, output_field_names):
class DaskUnionPlugin (line 26) | class DaskUnionPlugin(BaseRelPlugin):
method convert (line 34) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
FILE: dask_sql/physical/rel/logical/values.py
class DaskValuesPlugin (line 15) | class DaskValuesPlugin(BaseRelPlugin):
method convert (line 31) | def convert(
FILE: dask_sql/physical/rel/logical/window.py
class OverOperation (line 24) | class OverOperation:
method __call__ (line 25) | def __call__(self, partitioned_group, *args) -> pd.Series:
class FirstValueOperation (line 30) | class FirstValueOperation(OverOperation):
method call (line 31) | def call(self, partitioned_group, value_col):
class LastValueOperation (line 35) | class LastValueOperation(OverOperation):
method call (line 36) | def call(self, partitioned_group, value_col):
class SumOperation (line 40) | class SumOperation(OverOperation):
method call (line 41) | def call(self, partitioned_group, value_col):
class CountOperation (line 45) | class CountOperation(OverOperation):
method call (line 46) | def call(self, partitioned_group, value_col=None):
class MaxOperation (line 53) | class MaxOperation(OverOperation):
method call (line 54) | def call(self, partitioned_group, value_col):
class MinOperation (line 58) | class MinOperation(OverOperation):
method call (line 59) | def call(self, partitioned_group, value_col):
class AvgOperation (line 63) | class AvgOperation(OverOperation):
method call (line 64) | def call(self, partitioned_group, value_col):
class BoundDescription (line 68) | class BoundDescription(
function to_bound_description (line 82) | def to_bound_description(
class Indexer (line 96) | class Indexer(BaseIndexer):
method __init__ (line 102) | def __init__(self, start: int, end: int):
method _get_window_bounds (line 105) | def _get_window_bounds(
method get_window_bounds (line 134) | def get_window_bounds(
function map_on_each_group (line 145) | def map_on_each_group(
class DaskWindowPlugin (line 201) | class DaskWindowPlugin(BaseRelPlugin):
method convert (line 227) | def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> ...
method _apply_window (line 245) | def _apply_window(
method _extract_groupby (line 350) | def _extract_groupby(
method _extract_ordering (line 374) | def _extract_ordering(
method _extract_operations (line 391) | def _extract_operations(
FILE: dask_sql/physical/rex/base.py
class BaseRexPlugin (line 15) | class BaseRexPlugin:
method convert (line 26) | def convert(
FILE: dask_sql/physical/rex/convert.py
class RexConverter (line 25) | class RexConverter(Pluggable):
method add_plugin_class (line 42) | def add_plugin_class(cls, plugin_class: BaseRexPlugin, replace=True):
method convert (line 48) | def convert(
FILE: dask_sql/physical/rex/core/alias.py
class RexAliasPlugin (line 14) | class RexAliasPlugin(BaseRexPlugin):
method convert (line 23) | def convert(
FILE: dask_sql/physical/rex/core/call.py
function as_timelike (line 43) | def as_timelike(op):
class Operation (line 58) | class Operation:
method op_needs_dc (line 74) | def op_needs_dc(op):
method op_needs_rex (line 78) | def op_needs_rex(op):
method op_needs_context (line 82) | def op_needs_context(op):
method op_needs_rel (line 86) | def op_needs_rel(op):
method __init__ (line 89) | def __init__(self, f: Callable):
method __call__ (line 93) | def __call__(self, *operands, **kwargs) -> SeriesOrScalar:
method of (line 97) | def of(self, op: "Operation") -> "Operation":
class PredicateBasedOperation (line 108) | class PredicateBasedOperation(Operation):
method __init__ (line 114) | def __init__(
method apply (line 122) | def apply(self, *operands, **kwargs):
class TensorScalarOperation (line 129) | class TensorScalarOperation(PredicateBasedOperation):
method __init__ (line 135) | def __init__(self, tensor_f: Callable, scalar_f: Callable = None):
class ReduceOperation (line 140) | class ReduceOperation(Operation):
method __init__ (line 143) | def __init__(self, operation: Callable, unary_operation: Callable = No...
method reduce (line 151) | def reduce(self, *operands, **kwargs):
class SQLDivisionOperator (line 165) | class SQLDivisionOperator(Operation):
method __init__ (line 176) | def __init__(self):
method div (line 179) | def div(self, lhs, rhs, rex=None):
class IntDivisionOperator (line 192) | class IntDivisionOperator(Operation):
method __init__ (line 199) | def __init__(self):
method div (line 202) | def div(self, lhs, rhs):
class CaseOperation (line 217) | class CaseOperation(Operation):
method __init__ (line 220) | def __init__(self):
method case (line 223) | def case(self, *operands) -> SeriesOrScalar:
class CastOperation (line 256) | class CastOperation(Operation):
method __init__ (line 261) | def __init__(self):
method cast (line 264) | def cast(self, operand, rex=None) -> SeriesOrScalar:
class IsFalseOperation (line 295) | class IsFalseOperation(Operation):
method __init__ (line 298) | def __init__(self):
method false_ (line 301) | def false_(
class IsTrueOperation (line 315) | class IsTrueOperation(Operation):
method __init__ (line 318) | def __init__(self):
method true_ (line 321) | def true_(
class NegativeOperation (line 335) | class NegativeOperation(Operation):
method __init__ (line 338) | def __init__(self):
method negative_ (line 341) | def negative_(
class NotOperation (line 348) | class NotOperation(Operation):
method __init__ (line 351) | def __init__(self):
method not_ (line 354) | def not_(
class IsNullOperation (line 367) | class IsNullOperation(Operation):
method __init__ (line 370) | def __init__(self):
method null (line 373) | def null(
class IsNotDistinctOperation (line 386) | class IsNotDistinctOperation(Operation):
method __init__ (line 389) | def __init__(self):
method not_distinct (line 392) | def not_distinct(self, lhs: SeriesOrScalar, rhs: SeriesOrScalar) -> Se...
class RegexOperation (line 401) | class RegexOperation(Operation):
method __init__ (line 406) | def __init__(self):
method regex (line 409) | def regex(self, test: SeriesOrScalar, regex: str, rex=None) -> SeriesO...
class LikeOperation (line 471) | class LikeOperation(RegexOperation):
method __init__ (line 472) | def __init__(self, case_sensitive: bool = True):
class SimilarOperation (line 495) | class SimilarOperation(RegexOperation):
class PositionOperation (line 507) | class PositionOperation(Operation):
method __init__ (line 510) | def __init__(self):
method position (line 513) | def position(self, search, s, start=None):
class SubStringOperation (line 526) | class SubStringOperation(Operation):
method __init__ (line 529) | def __init__(self):
method substring (line 532) | def substring(self, s, start, length=None):
class TrimOperation (line 549) | class TrimOperation(Operation):
method __init__ (line 552) | def __init__(self, flag="BOTH"):
method trim (line 556) | def trim(self, s, search):
class ReplaceOperation (line 572) | class ReplaceOperation(Operation):
method __init__ (line 575) | def __init__(self):
method replace (line 578) | def replace(self, s, pat, repl):
class OverlayOperation (line 585) | class OverlayOperation(Operation):
method __init__ (line 588) | def __init__(self):
method overlay (line 591) | def overlay(self, s, replace, start, length=None):
class CoalesceOperation (line 609) | class CoalesceOperation(Operation):
method __init__ (line 610) | def __init__(self):
method coalesce (line 613) | def coalesce(self, *operands):
class ToTimestampOperation (line 628) | class ToTimestampOperation(Operation):
method __init__ (line 629) | def __init__(self):
method to_timestamp (line 632) | def to_timestamp(self, df, format):
class YearOperation (line 654) | class YearOperation(Operation):
method __init__ (line 655) | def __init__(self):
method extract_year (line 658) | def extract_year(self, df: SeriesOrScalar):
class TimeStampAddOperation (line 663) | class TimeStampAddOperation(Operation):
method __init__ (line 664) | def __init__(self):
method timestampadd (line 667) | def timestampadd(self, unit, interval, df: SeriesOrScalar):
class DatetimeSubOperation (line 710) | class DatetimeSubOperation(Operation):
method __init__ (line 716) | def __init__(self):
method datetime_sub (line 719) | def datetime_sub(self, unit, df1, df2):
class CeilFloorOperation (line 767) | class CeilFloorOperation(PredicateBasedOperation):
method __init__ (line 772) | def __init__(self, round_method: str):
method _round_datetime (line 786) | def _round_datetime(self, *operands):
class BaseRandomOperation (line 809) | class BaseRandomOperation(Operation):
method random_function (line 822) | def random_function(self, partition, random_state, kwargs):
method random_frame (line 826) | def random_frame(self, seed: int, dc: DataContainer, **kwargs) -> dd.S...
class RandOperation (line 852) | class RandOperation(BaseRandomOperation):
method __init__ (line 855) | def __init__(self):
method rand (line 858) | def rand(self, seed: int = None, dc: DataContainer = None):
method random_function (line 861) | def random_function(self, partition, random_state, kwargs):
class RandIntegerOperation (line 865) | class RandIntegerOperation(BaseRandomOperation):
method __init__ (line 868) | def __init__(self):
method rand_integer (line 871) | def rand_integer(
method random_function (line 880) | def random_function(self, partition, random_state, kwargs):
class SearchOperation (line 884) | class SearchOperation(Operation):
method __init__ (line 894) | def __init__(self):
method search (line 897) | def search(self, series: dd.Series, sarg: SargPythonImplementation):
class ExtractOperation (line 909) | class ExtractOperation(Operation):
method __init__ (line 914) | def __init__(self):
method date_part (line 917) | def date_part(self, what, df: SeriesOrScalar):
class BetweenOperation (line 963) | class BetweenOperation(Operation):
method __init__ (line 970) | def __init__(self):
method between (line 973) | def between(self, series: dd.Series, low, high, rex=None):
class InListOperation (line 981) | class InListOperation(Operation):
method __init__ (line 988) | def __init__(self):
method inList (line 991) | def inList(self, series: dd.Series, *operands, rex=None):
class InSubqueryOperation (line 996) | class InSubqueryOperation(Operation):
method __init__ (line 1005) | def __init__(self):
method inSubquery (line 1008) | def inSubquery(
class RexCallPlugin (line 1029) | class RexCallPlugin(BaseRexPlugin):
method convert (line 1158) | def convert(
FILE: dask_sql/physical/rex/core/input_ref.py
class RexInputRefPlugin (line 13) | class RexInputRefPlugin(BaseRexPlugin):
method convert (line 22) | def convert(
FILE: dask_sql/physical/rex/core/literal.py
class SargPythonImplementation (line 20) | class SargPythonImplementation:
class Range (line 28) | class Range:
method filter_on (line 52) | def filter_on(self, series: dd.Series):
method __repr__ (line 69) | def __repr__(self) -> str:
method __repr__ (line 78) | def __repr__(self) -> str:
class RexLiteralPlugin (line 82) | class RexLiteralPlugin(BaseRexPlugin):
method convert (line 94) | def convert(
FILE: dask_sql/physical/rex/core/subquery.py
class RexScalarSubqueryPlugin (line 14) | class RexScalarSubqueryPlugin(BaseRexPlugin):
method convert (line 23) | def convert(
FILE: dask_sql/physical/utils/filter.py
function attempt_predicate_pushdown (line 17) | def attempt_predicate_pushdown(
class DNF (line 167) | class DNF:
class _Or (line 170) | class _Or(frozenset):
method to_list_tuple (line 173) | def to_list_tuple(self) -> list:
class _And (line 187) | class _And(frozenset):
method to_list_tuple (line 190) | def to_list_tuple(self) -> list:
method __init__ (line 199) | def __init__(self, filters: DNF | _And | _Or | list | tuple | None) ->...
method to_list_tuple (line 205) | def to_list_tuple(self) -> list:
method __bool__ (line 208) | def __bool__(self) -> bool:
method normalize (line 212) | def normalize(cls, filters: _And | _Or | list | tuple | None):
method combine (line 252) | def combine(self, other: DNF | _And | _Or | list | tuple | None) -> DNF:
function _preprocess_layers (line 317) | def _preprocess_layers(input_layers):
class RegenerableLayer (line 344) | class RegenerableLayer:
method __init__ (line 353) | def __init__(self, layer, creation_info):
method _regenerate_collection (line 357) | def _regenerate_collection(
method _dnf_filter_expression (line 410) | def _dnf_filter_expression(self, dsk):
class RegenerableGraph (line 435) | class RegenerableGraph:
method __init__ (line 443) | def __init__(self, layers: dict):
method from_hlg (line 447) | def from_hlg(cls, hlg: HighLevelGraph):
function _get_blockwise_input (line 492) | def _get_blockwise_input(input_index, indices: list, dsk: RegenerableGra...
function _inv (line 501) | def _inv(symbol: str):
function _blockwise_comparison_dnf (line 514) | def _blockwise_comparison_dnf(op, indices: list, dsk: RegenerableGraph) ...
function _blockwise_logical_dnf (line 528) | def _blockwise_logical_dnf(op, indices: list, dsk: RegenerableGraph) -> ...
function _blockwise_getitem_dnf (line 547) | def _blockwise_getitem_dnf(op, indices: list, dsk: RegenerableGraph):
function _blockwise_pass_through_dnf (line 553) | def _blockwise_pass_through_dnf(op, indices: list, dsk: RegenerableGraph):
function _blockwise_isin_dnf (line 558) | def _blockwise_isin_dnf(op, indices: list, dsk: RegenerableGraph) -> DNF:
function _blockwise_isna_dnf (line 565) | def _blockwise_isna_dnf(op, indices: list, dsk: RegenerableGraph) -> DNF:
function _blockwise_inv_dnf (line 571) | def _blockwise_inv_dnf(op, indices: list, dsk: RegenerableGraph) -> DNF:
FILE: dask_sql/physical/utils/groupby.py
function get_groupby_with_nulls_cols (line 6) | def get_groupby_with_nulls_cols(
FILE: dask_sql/physical/utils/ml_classes.py
function get_cpu_classes (line 1) | def get_cpu_classes():
function get_gpu_classes (line 16) | def get_gpu_classes():
function add_boosting_classes (line 104) | def add_boosting_classes(my_classes):
FILE: dask_sql/physical/utils/sort.py
function apply_sort (line 9) | def apply_sort(
function topk_sort (line 78) | def topk_sort(
function sort_partition_func (line 90) | def sort_partition_func(
function is_topk_optimizable (line 120) | def is_topk_optimizable(
FILE: dask_sql/physical/utils/statistics.py
function parquet_statistics (line 21) | def parquet_statistics(
function _read_partition_stats_group (line 145) | def _read_partition_stats_group(parts, fs, engine, columns=None):
FILE: dask_sql/server/app.py
function empty (line 20) | async def empty(request: Request):
function cancel (line 29) | async def cancel(uuid: str, request: Request):
function status (line 45) | async def status(uuid: str, request: Request):
function query (line 70) | async def query(request: Request):
function run_server (line 103) | def run_server(
function main (line 210) | def main(): # pragma: no cover
function _init_app (line 268) | def _init_app(
FILE: dask_sql/server/presto_jdbc.py
function create_meta_data (line 10) | def create_meta_data(c: Context):
function create_catalog_row (line 93) | def create_catalog_row(catalog: str = ""):
function create_schema_row (line 97) | def create_schema_row(catalog: str = "", schema: str = ""):
function create_table_row (line 101) | def create_table_row(catalog: str = "", schema: str = "", table: str = ""):
function create_column_row (line 116) | def create_column_row(
FILE: dask_sql/server/responses.py
class StageStats (line 11) | class StageStats:
method __init__ (line 12) | def __init__(self):
class StatementStats (line 28) | class StatementStats:
method __init__ (line 29) | def __init__(self):
class QueryResults (line 51) | class QueryResults:
method __init__ (line 52) | def __init__(self, request: Request, next_url: str = None, cancel_url:...
class DataResults (line 65) | class DataResults(QueryResults):
method get_column_description (line 67) | def get_column_description(df):
method get_data_description (line 85) | def get_data_description(df):
method convert_cell (line 94) | def convert_cell(cell):
method convert_row (line 115) | def convert_row(row):
method __init__ (line 118) | def __init__(self, df: dd.DataFrame, request: Request):
class ErrorResults (line 128) | class ErrorResults(QueryResults):
method __init__ (line 129) | def __init__(self, error: Exception, request: Request):
class QueryError (line 135) | class QueryError:
method __init__ (line 136) | def __init__(self, error: Exception):
FILE: dask_sql/utils.py
function is_frame (line 19) | def is_frame(df):
function is_datetime (line 31) | def is_datetime(obj):
function convert_to_datetime (line 38) | def convert_to_datetime(df):
function is_cudf_type (line 49) | def is_cudf_type(obj):
class Pluggable (line 61) | class Pluggable:
method add_plugin (line 73) | def add_plugin(cls, names, plugin, replace=True):
method get_plugin (line 84) | def get_plugin(cls, name):
method get_plugins (line 89) | def get_plugins(cls):
class ParsingException (line 94) | class ParsingException(Exception):
method __init__ (line 100) | def __init__(self, sql, validation_exception_string):
class OptimizationException (line 108) | class OptimizationException(Exception):
method __init__ (line 114) | def __init__(self, exception_string):
class LoggableDataFrame (line 121) | class LoggableDataFrame:
method __init__ (line 124) | def __init__(self, df):
method __str__ (line 127) | def __str__(self):
function convert_sql_kwargs (line 144) | def convert_sql_kwargs(
function import_class (line 181) | def import_class(name: str) -> type:
function new_temporary_column (line 191) | def new_temporary_column(df: dd.DataFrame) -> str:
FILE: src/dialect.rs
type DaskDialect (line 12) | pub struct DaskDialect {}
method is_identifier_start (line 15) | fn is_identifier_start(&self, ch: char) -> bool {
method is_identifier_part (line 22) | fn is_identifier_part(&self, ch: char) -> bool {
method is_delimited_identifier_start (line 35) | fn is_delimited_identifier_start(&self, ch: char) -> bool {
method is_proper_identifier_inside_quotes (line 39) | fn is_proper_identifier_inside_quotes(&self, mut _chars: Peekable<Chars<...
method supports_filter_during_aggregation (line 43) | fn supports_filter_during_aggregation(&self) -> bool {
method parse_prefix (line 48) | fn parse_prefix(&self, parser: &mut Parser) -> Option<Result<Expr, Parse...
FILE: src/error.rs
type Result (line 9) | pub type Result<T> = std::result::Result<T, DaskPlannerError>;
type DaskPlannerError (line 12) | pub enum DaskPlannerError {
method from (line 33) | fn from(err: TokenizerError) -> Self {
method from (line 39) | fn from(err: ParserError) -> Self {
method from (line 45) | fn from(err: DataFusionError) -> Self {
method fmt (line 21) | fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
method from (line 51) | fn from(err: DaskPlannerError) -> PyErr {
FILE: src/expression.rs
type PyExpr (line 49) | pub struct PyExpr {
method from (line 92) | pub fn from(expr: Expr, input: Option<Vec<Arc<LogicalPlan>>>) -> PyExpr {
method _column_name (line 100) | pub fn _column_name(&self, plan: &LogicalPlan) -> Result<String> {
method _rex_type (line 105) | fn _rex_type(&self, expr: &Expr) -> RexType {
method literal (line 159) | pub fn literal(value: PyScalarValue) -> PyExpr {
method subquery_plan (line 166) | pub fn subquery_plan(&self) -> PyResult<logical::PyLogicalPlan> {
method is_input_reference (line 183) | pub fn is_input_reference(&self) -> PyResult<bool> {
method to_string (line 188) | pub fn to_string(&self) -> PyResult<String> {
method index (line 194) | pub fn index(&self) -> PyResult<usize> {
method get_expr_type (line 270) | pub fn get_expr_type(&self) -> PyResult<String> {
method rex_type (line 319) | pub fn rex_type(&self) -> PyResult<RexType> {
method column_name (line 324) | pub fn column_name(&self, mut plan: logical::PyLogicalPlan) -> PyResul...
method get_operands (line 334) | pub fn get_operands(&self) -> PyResult<Vec<PyExpr>> {
method get_operator_name (line 459) | pub fn get_operator_name(&self) -> PyResult<String> {
method get_type (line 512) | pub fn get_type(&self) -> PyResult<String> {
method get_precision_scale (line 650) | pub fn get_precision_scale(&self) -> PyResult<(u8, i8)> {
method get_filter_expr (line 672) | pub fn get_filter_expr(&self) -> PyResult<Option<PyExpr>> {
method float_32_value (line 699) | pub fn float_32_value(&self) -> PyResult<Option<f32>> {
method float_64_value (line 704) | pub fn float_64_value(&self) -> PyResult<Option<f64>> {
method decimal_128_value (line 709) | pub fn decimal_128_value(&mut self) -> PyResult<(Option<i128>, u8, i8)> {
method int_8_value (line 717) | pub fn int_8_value(&self) -> PyResult<Option<i8>> {
method int_16_value (line 722) | pub fn int_16_value(&self) -> PyResult<Option<i16>> {
method int_32_value (line 727) | pub fn int_32_value(&self) -> PyResult<Option<i32>> {
method int_64_value (line 732) | pub fn int_64_value(&self) -> PyResult<Option<i64>> {
method uint_8_value (line 737) | pub fn uint_8_value(&self) -> PyResult<Option<u8>> {
method uint_16_value (line 742) | pub fn uint_16_value(&self) -> PyResult<Option<u16>> {
method uint_32_value (line 747) | pub fn uint_32_value(&self) -> PyResult<Option<u32>> {
method uint_64_value (line 752) | pub fn uint_64_value(&self) -> PyResult<Option<u64>> {
method date_32_value (line 757) | pub fn date_32_value(&self) -> PyResult<Option<i32>> {
method date_64_value (line 762) | pub fn date_64_value(&self) -> PyResult<Option<i64>> {
method time_64_value (line 767) | pub fn time_64_value(&self) -> PyResult<Option<i64>> {
method timestamp_value (line 772) | pub fn timestamp_value(&mut self) -> PyResult<(Option<i64>, Option<Str...
method bool_value (line 786) | pub fn bool_value(&self) -> PyResult<Option<bool>> {
method string_value (line 791) | pub fn string_value(&self) -> PyResult<Option<String>> {
method interval_day_time_value (line 799) | pub fn interval_day_time_value(&self) -> PyResult<Option<(i32, i32)>> {
method interval_month_day_nano_value (line 813) | pub fn interval_month_day_nano_value(&self) -> PyResult<Option<(i32, i...
method is_negated (line 828) | pub fn is_negated(&self) -> PyResult<bool> {
method is_distinct_aggregation (line 842) | pub fn is_distinct_aggregation(&self) -> PyResult<bool> {
method is_sort_ascending (line 862) | pub fn is_sort_ascending(&self) -> PyResult<bool> {
method is_sort_nulls_first (line 874) | pub fn is_sort_nulls_first(&self) -> PyResult<bool> {
method get_escape_char (line 886) | pub fn get_escape_char(&self) -> PyResult<Option<char>> {
method get_scalar_value (line 902) | fn get_scalar_value(&self) -> Result<&ScalarValue> {
method from (line 56) | fn from(expr: PyExpr) -> Expr {
type PyScalarValue (line 63) | pub struct PyScalarValue {
method from (line 74) | fn from(scalar_value: ScalarValue) -> PyScalarValue {
method from (line 68) | fn from(pyscalar: PyScalarValue) -> ScalarValue {
function py_expr_list (line 80) | pub fn py_expr_list(input: &Arc<LogicalPlan>, expr: &[Expr]) -> PyResult...
function unexpected_literal_value (line 912) | fn unexpected_literal_value(value: &ScalarValue) -> PyErr {
function get_expr_name (line 916) | fn get_expr_name(expr: &Expr) -> Result<String> {
function expr_to_field (line 928) | pub fn expr_to_field(expr: &Expr, input_plan: &LogicalPlan) -> Result<DF...
function get_value_u32 (line 958) | fn get_value_u32() -> Result<()> {
function get_value_utf8 (line 964) | fn get_value_utf8() -> Result<()> {
function get_value_non_literal (line 970) | fn get_value_non_literal() -> Result<()> {
function test_get_value (line 982) | fn test_get_value(value: ScalarValue) -> Result<()> {
FILE: src/lib.rs
function _datafusion_lib (line 15) | fn _datafusion_lib(py: Python, m: &PyModule) -> PyResult<()> {
FILE: src/parser.rs
type CustomExpr (line 27) | pub enum CustomExpr {
type PySqlArg (line 35) | pub struct PySqlArg {
method new (line 41) | pub fn new(expr: Option<Expr>, custom: Option<CustomExpr>) -> Self {
method expected (line 45) | fn expected<T>(&self, expected: &str) -> PyResult<T> {
method is_collection (line 61) | pub fn is_collection(&self) -> PyResult<bool> {
method is_kwargs (line 72) | pub fn is_kwargs(&self) -> PyResult<bool> {
method get_operand_list (line 77) | pub fn get_operand_list(&self) -> PyResult<Vec<PySqlArg>> {
method get_kwargs (line 101) | pub fn get_kwargs(&self) -> PyResult<Vec<(String, PySqlArg)>> {
method get_sql_type (line 109) | pub fn get_sql_type(&self) -> PyResult<SqlTypeName> {
method get_sql_value (line 139) | pub fn get_sql_value(&self) -> PyResult<String> {
type CreateModel (line 166) | pub struct CreateModel {
type CreateExperiment (line 182) | pub struct CreateExperiment {
type PredictModel (line 198) | pub struct PredictModel {
type CreateCatalogSchema (line 208) | pub struct CreateCatalogSchema {
type CreateTable (line 219) | pub struct CreateTable {
type DropModel (line 233) | pub struct DropModel {
type ExportModel (line 243) | pub struct ExportModel {
type DescribeModel (line 253) | pub struct DescribeModel {
type ShowSchemas (line 261) | pub struct ShowSchemas {
type ShowTables (line 270) | pub struct ShowTables {
type ShowColumns (line 278) | pub struct ShowColumns {
type ShowModels (line 286) | pub struct ShowModels {
type DropSchema (line 292) | pub struct DropSchema {
type UseSchema (line 301) | pub struct UseSchema {
type AnalyzeTable (line 308) | pub struct AnalyzeTable {
type AlterTable (line 318) | pub struct AlterTable {
type AlterSchema (line 327) | pub struct AlterSchema {
type DaskStatement (line 336) | pub enum DaskStatement {
type DaskParser (line 376) | pub struct DaskParser<'a> {
function new (line 383) | pub fn new(sql: &str) -> Result<Self, ParserError> {
function new_with_dialect (line 389) | pub fn new_with_dialect(sql: &str, dialect: &'a dyn Dialect) -> Result<S...
function parse_sql (line 400) | pub fn parse_sql(sql: &str) -> Result<VecDeque<DaskStatement>, ParserErr...
function parse_sql_with_dialect (line 406) | pub fn parse_sql_with_dialect(
function expected (line 434) | fn expected<T>(&self, expected: &str, found: TokenWithLocation) -> Resul...
function parse_statement (line 442) | pub fn parse_statement(&mut self) -> Result<DaskStatement, ParserError> {
function parse_create (line 552) | pub fn parse_create(&mut self) -> Result<DaskStatement, ParserError> {
function parse_drop (line 637) | pub fn parse_drop(&mut self) -> Result<DaskStatement, ParserError> {
function parse_show (line 680) | pub fn parse_show(&mut self) -> Result<DaskStatement, ParserError> {
function parse_describe (line 745) | pub fn parse_describe(&mut self) -> Result<DaskStatement, ParserError> {
function parse_use (line 772) | pub fn parse_use(&mut self) -> Result<DaskStatement, ParserError> {
function parse_analyze (line 799) | pub fn parse_analyze(&mut self) -> Result<DaskStatement, ParserError> {
function parse_alter (line 827) | pub fn parse_alter(&mut self) -> Result<DaskStatement, ParserError> {
function parse_predict_model (line 857) | pub fn parse_predict_model(&mut self) -> Result<DaskStatement, ParserErr...
function parse_create_model (line 901) | fn parse_create_model(
function parse_comma_separated (line 946) | fn parse_comma_separated<T, F>(&mut self, mut f: F) -> Result<Vec<T>, Pa...
function parse_key_value_pair (line 960) | fn parse_key_value_pair(&mut self) -> Result<(String, PySqlArg), ParserE...
function parse_create_experiment (line 1004) | fn parse_create_experiment(
function parse_create_schema (line 1049) | fn parse_create_schema(
function parse_create_table (line 1070) | fn parse_create_table(
function parse_export_model (line 1154) | fn parse_export_model(&mut self) -> Result<DaskStatement, ParserError> {
function parse_drop_model (line 1184) | fn parse_drop_model(&mut self) -> Result<DaskStatement, ParserError> {
function parse_describe_model (line 1199) | fn parse_describe_model(&mut self) -> Result<DaskStatement, ParserError> {
function parse_show_schemas (line 1212) | fn parse_show_schemas(&mut self) -> Result<DaskStatement, ParserError> {
function parse_show_tables (line 1251) | fn parse_show_tables(&mut self) -> Result<DaskStatement, ParserError> {
function parse_show_columns (line 1266) | fn parse_show_columns(&mut self) -> Result<DaskStatement, ParserError> {
function parse_show_models (line 1277) | fn parse_show_models(&mut self) -> Result<DaskStatement, ParserError> {
function parse_analyze_table (line 1289) | fn parse_analyze_table(&mut self) -> Result<DaskStatement, ParserError> {
function parse_alter_table (line 1329) | fn parse_alter_table(&mut self) -> Result<DaskStatement, ParserError> {
function parse_alter_schema (line 1350) | fn parse_alter_schema(&mut self) -> Result<DaskStatement, ParserError> {
function timestampadd (line 1372) | fn timestampadd() {
function to_timestamp (line 1382) | fn to_timestamp() {
function create_model (line 1401) | fn create_model() {
FILE: src/sql.rs
type DaskSQLContext (line 85) | pub struct DaskSQLContext {
method new (line 511) | pub fn new(
method set_optimizer_config (line 525) | pub fn set_optimizer_config(&mut self, config: DaskSQLOptimizerConfig)...
method use_schema (line 531) | pub fn use_schema(&mut self, schema_name: &str) -> PyResult<()> {
method register_schema (line 543) | pub fn register_schema(
method register_table (line 553) | pub fn register_table(
method parse_sql (line 570) | pub fn parse_sql(&self, sql: &str) -> PyResult<Vec<statement::PyStatem...
method logical_relational_algebra (line 586) | pub fn logical_relational_algebra(
method run_preoptimizer (line 598) | pub fn run_preoptimizer(
method optimize_relational_algebra (line 612) | pub fn optimize_relational_algebra(
method _logical_relational_algebra (line 668) | pub fn _logical_relational_algebra(
type DaskSQLOptimizerConfig (line 95) | pub struct DaskSQLOptimizerConfig {
method new (line 106) | pub fn new(
method get_table_provider (line 124) | fn get_table_provider(
method get_function_meta (line 198) | fn get_function_meta(&self, name: &str) -> Option<Arc<ScalarUDF>> {
method get_aggregate_meta (line 405) | fn get_aggregate_meta(&self, name: &str) -> Option<Arc<AggregateUDF>> {
method get_variable_type (line 492) | fn get_variable_type(&self, _: &[String]) -> Option<DataType> {
method options (line 496) | fn options(&self) -> &ConfigOptions {
method get_window_meta (line 500) | fn get_window_meta(
type OptimizablePlanVisitor (line 818) | pub struct OptimizablePlanVisitor;
type N (line 821) | type N = LogicalPlan;
method pre_visit (line 823) | fn pre_visit(&mut self, plan: &LogicalPlan) -> Result<VisitRecursion, Da...
method post_visit (line 831) | fn post_visit(&mut self, _plan: &LogicalPlan) -> Result<VisitRecursion, ...
function generate_signatures (line 836) | fn generate_signatures(cartesian_setup: Vec<Vec<DataType>>) -> Signature {
function test_generate_signatures (line 877) | fn test_generate_signatures() {
FILE: src/sql/column.rs
type PyColumn (line 6) | pub struct PyColumn {
method from (line 18) | fn from(column: Column) -> PyColumn {
method relation (line 26) | pub fn relation(&self) -> String {
method name (line 31) | pub fn name(&self) -> String {
method from (line 12) | fn from(column: PyColumn) -> Column {
FILE: src/sql/exceptions.rs
function py_type_err (line 11) | pub fn py_type_err(e: impl Debug) -> PyErr {
function py_runtime_err (line 15) | pub fn py_runtime_err(e: impl Debug) -> PyErr {
function py_parsing_exp (line 19) | pub fn py_parsing_exp(e: impl Debug) -> PyErr {
function py_optimization_exp (line 23) | pub fn py_optimization_exp(e: impl Debug) -> PyErr {
FILE: src/sql/function.rs
type DaskFunction (line 10) | pub struct DaskFunction {
method new (line 18) | pub fn new(
method add_type_mapping (line 33) | pub fn add_type_mapping(&mut self, input_types: Vec<PyDataType>, retur...
FILE: src/sql/logical.rs
type PyLogicalPlan (line 67) | pub struct PyLogicalPlan {
method current_node (line 78) | pub(crate) fn current_node(&mut self) -> LogicalPlan {
method aggregate (line 102) | pub fn aggregate(&self) -> PyResult<aggregate::PyAggregate> {
method empty_relation (line 107) | pub fn empty_relation(&self) -> PyResult<empty_relation::PyEmptyRelati...
method explain (line 112) | pub fn explain(&self) -> PyResult<explain::PyExplain> {
method filter (line 117) | pub fn filter(&self) -> PyResult<filter::PyFilter> {
method join (line 122) | pub fn join(&self) -> PyResult<join::PyJoin> {
method limit (line 127) | pub fn limit(&self) -> PyResult<limit::PyLimit> {
method projection (line 132) | pub fn projection(&self) -> PyResult<projection::PyProjection> {
method sort (line 137) | pub fn sort(&self) -> PyResult<sort::PySort> {
method subquery_alias (line 142) | pub fn subquery_alias(&self) -> PyResult<subquery_alias::PySubqueryAli...
method window (line 147) | pub fn window(&self) -> PyResult<window::PyWindow> {
method table_scan (line 152) | pub fn table_scan(&self) -> PyResult<table_scan::PyTableScan> {
method create_memory_table (line 157) | pub fn create_memory_table(&self) -> PyResult<create_memory_table::PyC...
method create_model (line 162) | pub fn create_model(&self) -> PyResult<create_model::PyCreateModel> {
method create_experiment (line 167) | pub fn create_experiment(&self) -> PyResult<create_experiment::PyCreat...
method drop_table (line 172) | pub fn drop_table(&self) -> PyResult<drop_table::PyDropTable> {
method drop_model (line 177) | pub fn drop_model(&self) -> PyResult<drop_model::PyDropModel> {
method show_schemas (line 182) | pub fn show_schemas(&self) -> PyResult<show_schemas::PyShowSchema> {
method repartition_by (line 187) | pub fn repartition_by(&self) -> PyResult<repartition_by::PyRepartition...
method show_tables (line 192) | pub fn show_tables(&self) -> PyResult<show_tables::PyShowTables> {
method create_table (line 197) | pub fn create_table(&self) -> PyResult<create_table::PyCreateTable> {
method predict_model (line 202) | pub fn predict_model(&self) -> PyResult<predict_model::PyPredictModel> {
method describe_model (line 207) | pub fn describe_model(&self) -> PyResult<describe_model::PyDescribeMod...
method export_model (line 212) | pub fn export_model(&self) -> PyResult<export_model::PyExportModel> {
method show_columns (line 217) | pub fn show_columns(&self) -> PyResult<show_columns::PyShowColumns> {
method show_models (line 221) | pub fn show_models(&self) -> PyResult<show_models::PyShowModels> {
method analyze_table (line 226) | pub fn analyze_table(&self) -> PyResult<analyze_table::PyAnalyzeTable> {
method create_catalog_schema (line 231) | pub fn create_catalog_schema(&self) -> PyResult<create_catalog_schema:...
method drop_schema (line 236) | pub fn drop_schema(&self) -> PyResult<drop_schema::PyDropSchema> {
method use_schema (line 241) | pub fn use_schema(&self) -> PyResult<use_schema::PyUseSchema> {
method alter_table (line 246) | pub fn alter_table(&self) -> PyResult<alter_table::PyAlterTable> {
method alter_schema (line 251) | pub fn alter_schema(&self) -> PyResult<alter_schema::PyAlterSchema> {
method get_inputs (line 256) | pub fn get_inputs(&mut self) -> PyResult<Vec<PyLogicalPlan>> {
method table (line 267) | pub fn table(&mut self) -> PyResult<table::DaskTable> {
method get_current_node_schema_name (line 277) | pub fn get_current_node_schema_name(&self) -> PyResult<&str> {
method get_current_node_table_name (line 292) | pub fn get_current_node_table_name(&mut self) -> PyResult<String> {
method get_current_node_type (line 300) | pub fn get_current_node_type(&mut self) -> PyResult<&str> {
method explain_original (line 380) | pub fn explain_original(&self) -> PyResult<String> {
method explain_current (line 385) | pub fn explain_current(&mut self) -> PyResult<String> {
method row_type (line 390) | pub fn row_type(&self) -> PyResult<RelDataType> {
method from (line 446) | fn from(logical_plan: LogicalPlan) -> PyLogicalPlan {
function to_py_plan (line 90) | fn to_py_plan<T: TryFrom<LogicalPlan, Error = PyErr>>(
method from (line 440) | fn from(logical_plan: PyLogicalPlan) -> LogicalPlan {
FILE: src/sql/logical/aggregate.rs
type PyAggregate (line 16) | pub struct PyAggregate {
method distinct_columns (line 25) | pub fn distinct_columns(&self) -> PyResult<Vec<String>> {
method group_expressions (line 36) | pub fn group_expressions(&self) -> PyResult<Vec<PyExpr>> {
method agg_expressions (line 45) | pub fn agg_expressions(&self) -> PyResult<Vec<PyExpr>> {
method agg_func_name (line 53) | pub fn agg_func_name(&self, expr: PyExpr) -> PyResult<String> {
method aggregation_arguments (line 58) | pub fn aggregation_arguments(&self, expr: PyExpr) -> PyResult<Vec<PyEx...
method distinct_agg_expr (line 63) | pub fn distinct_agg_expr(&self, expr: PyExpr) -> PyResult<bool> {
method distinct_node (line 68) | pub fn distinct_node(&self) -> PyResult<bool> {
method _aggregation_arguments (line 74) | fn _aggregation_arguments(&self, expr: &Expr) -> PyResult<Vec<PyExpr>> {
type Error (line 115) | type Error = PyErr;
method try_from (line 117) | fn try_from(logical_plan: LogicalPlan) -> Result<Self, Self::Error> {
function _agg_func_name (line 89) | fn _agg_func_name(expr: &Expr) -> PyResult<String> {
function _distinct_agg_expr (line 100) | fn _distinct_agg_expr(expr: &Expr) -> PyResult<bool> {
FILE: src/sql/logical/alter_schema.rs
type AlterSchemaPlanNode (line 22) | pub struct AlterSchemaPlanNode {
method fmt (line 29) | fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
method hash (line 35) | fn hash<H: Hasher>(&self, state: &mut H) {
method as_any (line 43) | fn as_any(&self) -> &dyn Any {
method inputs (line 47) | fn inputs(&self) -> Vec<&LogicalPlan> {
method schema (line 51) | fn schema(&self) -> &DFSchemaRef {
method expressions (line 55) | fn expressions(&self) -> Vec<Expr> {
method fmt_for_explain (line 62) | fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result {
method from_template (line 70) | fn from_template(
method name (line 82) | fn name(&self) -> &str {
method dyn_hash (line 86) | fn dyn_hash(&self, state: &mut dyn Hasher) {
method dyn_eq (line 91) | fn dyn_eq(&self, other: &dyn UserDefinedLogicalNode) -> bool {
type PyAlterSchema (line 100) | pub struct PyAlterSchema {
method get_old_schema_name (line 107) | fn get_old_schema_name(&self) -> PyResult<String> {
method get_new_schema_name (line 112) | fn get_new_schema_name(&self) -> PyResult<String> {
type Error (line 118) | type Error = PyErr;
method try_from (line 120) | fn try_from(logical_plan: logical::LogicalPlan) -> Result<Self, Self::...
FILE: src/sql/logical/alter_table.rs
type AlterTablePlanNode (line 22) | pub struct AlterTablePlanNode {
method fmt (line 31) | fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
method hash (line 37) | fn hash<H: Hasher>(&self, state: &mut H) {
method as_any (line 47) | fn as_any(&self) -> &dyn Any {
method inputs (line 51) | fn inputs(&self) -> Vec<&LogicalPlan> {
method schema (line 55) | fn schema(&self) -> &DFSchemaRef {
method expressions (line 59) | fn expressions(&self) -> Vec<Expr> {
method fmt_for_explain (line 66) | fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result {
method from_template (line 74) | fn from_template(
method name (line 88) | fn name(&self) -> &str {
method dyn_hash (line 92) | fn dyn_hash(&self, state: &mut dyn Hasher) {
method dyn_eq (line 97) | fn dyn_eq(&self, other: &dyn UserDefinedLogicalNode) -> bool {
type PyAlterTable (line 106) | pub struct PyAlterTable {
method get_old_table_name (line 113) | fn get_old_table_name(&self) -> PyResult<String> {
method get_new_table_name (line 118) | fn get_new_table_name(&self) -> PyResult<String> {
method get_schema_name (line 123) | fn get_schema_name(&self) -> PyResult<Option<String>> {
method get_if_exists (line 128) | fn get_if_exists(&self) -> PyResult<bool> {
type Error (line 134) | type Error = PyErr;
method try_from (line 136) | fn try_from(logical_plan: logical::LogicalPlan) -> Result<Self, Self::...
FILE: src/sql/logical/analyze_table.rs
type AnalyzeTablePlanNode (line 22) | pub struct AnalyzeTablePlanNode {
method fmt (line 30) | fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
method hash (line 36) | fn hash<H: Hasher>(&self, state: &mut H) {
method as_any (line 45) | fn as_any(&self) -> &dyn Any {
method inputs (line 49) | fn inputs(&self) -> Vec<&LogicalPlan> {
method schema (line 53) | fn schema(&self) -> &DFSchemaRef {
method expressions (line 57) | fn expressions(&self) -> Vec<Expr> {
method fmt_for_explain (line 64) | fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result {
method from_template (line 72) | fn from_template(
method name (line 85) | fn name(&self) -> &str {
method dyn_hash (line 89) | fn dyn_hash(&self, state: &mut dyn Hasher) {
method dyn_eq (line 94) | fn dyn_eq(&self, other: &dyn UserDefinedLogicalNode) -> bool {
type PyAnalyzeTable (line 103) | pub struct PyAnalyzeTable {
method get_table_name (line 110) | fn get_table_name(&self) -> PyResult<String> {
method get_schema_name (line 115) | fn get_schema_name(&self) -> PyResult<Option<String>> {
method get_columns (line 120) | fn get_columns(&self) -> PyResult<Vec<String>> {
type Error (line 126) | type Error = PyErr;
method try_from (line 128) | fn try_from(logical_plan: logical::LogicalPlan) -> Result<Self, Self::...
FILE: src/sql/logical/create_catalog_schema.rs
type CreateCatalogSchemaPlanNode (line 18) | pub struct CreateCatalogSchemaPlanNode {
method fmt (line 26) | fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
method hash (line 32) | fn hash<H: Hasher>(&self, state: &mut H) {
method as_any (line 41) | fn as_any(&self) -> &dyn Any {
method inputs (line 45) | fn inputs(&self) -> Vec<&LogicalPlan> {
method schema (line 49) | fn schema(&self) -> &DFSchemaRef {
method expressions (line 53) | fn expressions(&self) -> Vec<Expr> {
method fmt_for_explain (line 60) | fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result {
method from_template (line 68) | fn from_template(
method name (line 81) | fn name(&self) -> &str {
method dyn_hash (line 85) | fn dyn_hash(&self, state: &mut dyn Hasher) {
method dyn_eq (line 90) | fn dyn_eq(&self, other: &dyn UserDefinedLogicalNode) -> bool {
type PyCreateCatalogSchema (line 99) | pub struct PyCreateCatalogSchema {
method get_schema_name (line 106) | fn get_schema_name(&self) -> PyResult<String> {
method get_if_not_exists (line 111) | fn get_if_not_exists(&self) -> PyResult<bool> {
method get_replace (line 116) | fn get_replace(&self) -> PyResult<bool> {
type Error (line 122) | type Error = PyErr;
method try_from (line 124) | fn try_from(logical_plan: logical::LogicalPlan) -> Result<Self, Self::...
FILE: src/sql/logical/create_experiment.rs
type CreateExperimentPlanNode (line 21) | pub struct CreateExperimentPlanNode {
method fmt (line 31) | fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
method hash (line 37) | fn hash<H: Hasher>(&self, state: &mut H) {
method as_any (line 48) | fn as_any(&self) -> &dyn Any {
method inputs (line 52) | fn inputs(&self) -> Vec<&LogicalPlan> {
method schema (line 56) | fn schema(&self) -> &DFSchemaRef {
method expressions (line 60) | fn expressions(&self) -> Vec<Expr> {
method fmt_for_explain (line 67) | fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result {
method from_template (line 75) | fn from_template(
method name (line 91) | fn name(&self) -> &str {
method dyn_hash (line 95) | fn dyn_hash(&self, state: &mut dyn Hasher) {
method dyn_eq (line 100) | fn dyn_eq(&self, other: &dyn UserDefinedLogicalNode) -> bool {
type PyCreateExperiment (line 109) | pub struct PyCreateExperiment {
method get_select_query (line 119) | fn get_select_query(&self) -> PyResult<logical::PyLogicalPlan> {
method get_schema_name (line 124) | fn get_schema_name(&self) -> PyResult<Option<String>> {
method get_experiment_name (line 129) | fn get_experiment_name(&self) -> PyResult<String> {
method get_if_not_exists (line 134) | fn get_if_not_exists(&self) -> PyResult<bool> {
method get_or_replace (line 139) | pub fn get_or_replace(&self) -> PyResult<bool> {
method sql_with_options (line 144) | fn sql_with_options(&self) -> PyResult<Vec<(String, PySqlArg)>> {
type Error (line 150) | type Error = PyErr;
method try_from (line 152) | fn try_from(logical_plan: logical::LogicalPlan) -> Result<Self, Self::...
FILE: src/sql/logical/create_memory_table.rs
type PyCreateMemoryTable (line 12) | pub struct PyCreateMemoryTable {
method get_table_name (line 20) | pub fn get_table_name(&self) -> PyResult<String> {
method get_input (line 35) | pub fn get_input(&self) -> PyResult<PyLogicalPlan> {
method get_if_not_exists (line 56) | pub fn get_if_not_exists(&self) -> PyResult<bool> {
method get_or_replace (line 64) | pub fn get_or_replace(&self) -> PyResult<bool> {
method is_table (line 79) | pub fn is_table(&self) -> PyResult<bool> {
type Error (line 85) | type Error = PyErr;
method try_from (line 87) | fn try_from(logical_plan: LogicalPlan) -> Result<Self, Self::Error> {
FILE: src/sql/logical/create_model.rs
type CreateModelPlanNode (line 21) | pub struct CreateModelPlanNode {
method fmt (line 31) | fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
method hash (line 37) | fn hash<H: Hasher>(&self, state: &mut H) {
method as_any (line 48) | fn as_any(&self) -> &dyn Any {
method inputs (line 52) | fn inputs(&self) -> Vec<&LogicalPlan> {
method schema (line 56) | fn schema(&self) -> &DFSchemaRef {
method expressions (line 60) | fn expressions(&self) -> Vec<Expr> {
method fmt_for_explain (line 67) | fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result {
method from_template (line 71) | fn from_template(
method name (line 87) | fn name(&self) -> &str {
method dyn_hash (line 91) | fn dyn_hash(&self, state: &mut dyn Hasher) {
method dyn_eq (line 96) | fn dyn_eq(&self, other: &dyn UserDefinedLogicalNode) -> bool {
type PyCreateModel (line 105) | pub struct PyCreateModel {
method get_select_query (line 115) | fn get_select_query(&self) -> PyResult<logical::PyLogicalPlan> {
method get_schema_name (line 120) | fn get_schema_name(&self) -> PyResult<Option<String>> {
method get_model_name (line 125) | fn get_model_name(&self) -> PyResult<String> {
method get_if_not_exists (line 130) | fn get_if_not_exists(&self) -> PyResult<bool> {
method get_or_replace (line 135) | pub fn get_or_replace(&self) -> PyResult<bool> {
method sql_with_options (line 140) | fn sql_with_options(&self) -> PyResult<Vec<(String, PySqlArg)>> {
type Error (line 146) | type Error = PyErr;
method try_from (line 148) | fn try_from(logical_plan: logical::LogicalPlan) -> Result<Self, Self::...
FILE: src/sql/logical/create_table.rs
type CreateTablePlanNode (line 21) | pub struct CreateTablePlanNode {
method fmt (line 31) | fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
method hash (line 37) | fn hash<H: Hasher>(&self, state: &mut H) {
method as_any (line 48) | fn as_any(&self) -> &dyn Any {
method inputs (line 52) | fn inputs(&self) -> Vec<&LogicalPlan> {
method schema (line 56) | fn schema(&self) -> &DFSchemaRef {
method expressions (line 60) | fn expressions(&self) -> Vec<Expr> {
method fmt_for_explain (line 67) | fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result {
method from_template (line 71) | fn from_template(
method name (line 86) | fn name(&self) -> &str {
method dyn_hash (line 90) | fn dyn_hash(&self, state: &mut dyn Hasher) {
method dyn_eq (line 95) | fn dyn_eq(&self, other: &dyn UserDefinedLogicalNode) -> bool {
type PyCreateTable (line 104) | pub struct PyCreateTable {
method get_schema_name (line 111) | fn get_schema_name(&self) -> PyResult<Option<String>> {
method get_table_name (line 116) | fn get_table_name(&self) -> PyResult<String> {
method get_if_not_exists (line 121) | fn get_if_not_exists(&self) -> PyResult<bool> {
method get_or_replace (line 126) | fn get_or_replace(&self) -> PyResult<bool> {
method sql_with_options (line 131) | fn sql_with_options(&self) -> PyResult<Vec<(String, PySqlArg)>> {
type Error (line 137) | type Error = PyErr;
method try_from (line 139) | fn try_from(logical_plan: logical::LogicalPlan) -> Result<Self, Self::...
FILE: src/sql/logical/describe_model.rs
type DescribeModelPlanNode (line 18) | pub struct DescribeModelPlanNode {
method fmt (line 25) | fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
method hash (line 31) | fn hash<H: Hasher>(&self, state: &mut H) {
method as_any (line 39) | fn as_any(&self) -> &dyn Any {
method inputs (line 43) | fn inputs(&self) -> Vec<&LogicalPlan> {
method schema (line 47) | fn schema(&self) -> &DFSchemaRef {
method expressions (line 51) | fn expressions(&self) -> Vec<Expr> {
method fmt_for_explain (line 58) | fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result {
method from_template (line 62) | fn from_template(
method name (line 75) | fn name(&self) -> &str {
method dyn_hash (line 79) | fn dyn_hash(&self, state: &mut dyn Hasher) {
method dyn_eq (line 84) | fn dyn_eq(&self, other: &dyn UserDefinedLogicalNode) -> bool {
type PyDescribeModel (line 93) | pub struct PyDescribeModel {
method get_schema_name (line 100) | fn get_schema_name(&self) -> PyResult<Option<String>> {
method get_model_name (line 105) | fn get_model_name(&self) -> PyResult<String> {
type Error (line 111) | type Error = PyErr;
method try_from (line 113) | fn try_from(logical_plan: logical::LogicalPlan) -> Result<Self, Self::...
FILE: src/sql/logical/drop_model.rs
type DropModelPlanNode (line 18) | pub struct DropModelPlanNode {
method fmt (line 26) | fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
method hash (line 32) | fn hash<H: Hasher>(&self, state: &mut H) {
method as_any (line 41) | fn as_any(&self) -> &dyn Any {
method inputs (line 45) | fn inputs(&self) -> Vec<&LogicalPlan> {
method schema (line 49) | fn schema(&self) -> &DFSchemaRef {
method expressions (line 53) | fn expressions(&self) -> Vec<Expr> {
method fmt_for_explain (line 60) | fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result {
method from_template (line 64) | fn from_template(
method name (line 78) | fn name(&self) -> &str {
method dyn_hash (line 82) | fn dyn_hash(&self, state: &mut dyn Hasher) {
method dyn_eq (line 87) | fn dyn_eq(&self, other: &dyn UserDefinedLogicalNode) -> bool {
type PyDropModel (line 96) | pub struct PyDropModel {
method get_schema_name (line 103) | fn get_schema_name(&self) -> PyResult<Option<String>> {
method get_model_name (line 108) | fn get_model_name(&self) -> PyResult<String> {
method get_if_exists (line 113) | pub fn get_if_exists(&self) -> PyResult<bool> {
type Error (line 119) | type Error = PyErr;
method try_from (line 121) | fn try_from(logical_plan: logical::LogicalPlan) -> Result<Self, Self::...
FILE: src/sql/logical/drop_schema.rs
type DropSchemaPlanNode (line 18) | pub struct DropSchemaPlanNode {
method fmt (line 25) | fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
method hash (line 31) | fn hash<H: Hasher>(&self, state: &mut H) {
method as_any (line 39) | fn as_any(&self) -> &dyn Any {
method inputs (line 43) | fn inputs(&self) -> Vec<&LogicalPlan> {
method schema (line 47) | fn schema(&self) -> &DFSchemaRef {
method expressions (line 51) | fn expressions(&self) -> Vec<Expr> {
method fmt_for_explain (line 58) | fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result {
method from_template (line 62) | fn from_template(
method name (line 74) | fn name(&self) -> &str {
method dyn_hash (line 78) | fn dyn_hash(&self, state: &mut dyn Hasher) {
method dyn_eq (line 83) | fn dyn_eq(&self, other: &dyn UserDefinedLogicalNode) -> bool {
type PyDropSchema (line 92) | pub struct PyDropSchema {
method get_schema_name (line 99) | fn get_schema_name(&self) -> PyResult<String> {
method get_if_exists (line 104) | fn get_if_exists(&self) -> PyResult<bool> {
type Error (line 110) | type Error = PyErr;
method try_from (line 112) | fn try_from(logical_plan: logical::LogicalPlan) -> Result<Self, Self::...
FILE: src/sql/logical/drop_table.rs
type PyDropTable (line 11) | pub struct PyDropTable {
method get_name (line 18) | pub fn get_name(&self) -> PyResult<String> {
method get_if_exists (line 23) | pub fn get_if_exists(&self) -> PyResult<bool> {
type Error (line 29) | type Error = PyErr;
method try_from (line 31) | fn try_from(logical_plan: LogicalPlan) -> Result<Self, Self::Error> {
FILE: src/sql/logical/empty_relation.rs
type PyEmptyRelation (line 8) | pub struct PyEmptyRelation {
type Error (line 13) | type Error = PyErr;
method try_from (line 15) | fn try_from(logical_plan: LogicalPlan) -> Result<Self, Self::Error> {
method empty_column_names (line 31) | pub fn empty_column_names(&self) -> PyResult<Vec<String>> {
FILE: src/sql/logical/explain.rs
type PyExplain (line 8) | pub struct PyExplain {
method get_explain_string (line 16) | pub fn get_explain_string(&self) -> PyResult<Vec<String>> {
type Error (line 26) | type Error = PyErr;
method try_from (line 28) | fn try_from(logical_plan: LogicalPlan) -> Result<Self, Self::Error> {
FILE: src/sql/logical/export_model.rs
type ExportModelPlanNode (line 21) | pub struct ExportModelPlanNode {
method fmt (line 29) | fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
method hash (line 35) | fn hash<H: Hasher>(&self, state: &mut H) {
method as_any (line 44) | fn as_any(&self) -> &dyn Any {
method inputs (line 48) | fn inputs(&self) -> Vec<&LogicalPlan> {
method schema (line 52) | fn schema(&self) -> &DFSchemaRef {
method expressions (line 56) | fn expressions(&self) -> Vec<Expr> {
method fmt_for_explain (line 63) | fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result {
method from_template (line 67) | fn from_template(
method name (line 81) | fn name(&self) -> &str {
method dyn_hash (line 85) | fn dyn_hash(&self, state: &mut dyn Hasher) {
method dyn_eq (line 90) | fn dyn_eq(&self, other: &dyn UserDefinedLogicalNode) -> bool {
type PyExportModel (line 99) | pub struct PyExportModel {
method get_model_name (line 106) | fn get_model_name(&self) -> PyResult<String> {
method get_schema_name (line 111) | fn get_schema_name(&self) -> PyResult<Option<String>> {
method sql_with_options (line 116) | fn sql_with_options(&self) -> PyResult<Vec<(String, PySqlArg)>> {
type Error (line 122) | type Error = PyErr;
method try_from (line 124) | fn try_from(logical_plan: logical::LogicalPlan) -> Result<Self, Self::...
FILE: src/sql/logical/filter.rs
type PyFilter (line 8) | pub struct PyFilter {
method get_condition (line 16) | pub fn get_condition(&mut self) -> PyResult<PyExpr> {
type Error (line 25) | type Error = PyErr;
method try_from (line 27) | fn try_from(logical_plan: LogicalPlan) -> Result<Self, Self::Error> {
FILE: src/sql/logical/join.rs
type PyJoin (line 20) | pub struct PyJoin {
method join_condition (line 27) | pub fn join_condition(&self) -> PyResult<Option<PyExpr>> {
method join_conditions (line 74) | pub fn join_conditions(&mut self) -> PyResult<Vec<(column::PyColumn, c...
method join_type (line 107) | pub fn join_type(&mut self) -> PyResult<String> {
type Error (line 122) | type Error = PyErr;
method try_from (line 124) | fn try_from(logical_plan: LogicalPlan) -> Result<Self, Self::Error> {
FILE: src/sql/logical/limit.rs
type PyLimit (line 11) | pub struct PyLimit {
method skip (line 19) | pub fn skip(&self) -> PyResult<PyExpr> {
method fetch (line 28) | pub fn fetch(&self) -> PyResult<PyExpr> {
type Error (line 39) | type Error = PyErr;
method try_from (line 41) | fn try_from(logical_plan: LogicalPlan) -> Result<Self, Self::Error> {
FILE: src/sql/logical/predict_model.rs
type PredictModelPlanNode (line 19) | pub struct PredictModelPlanNode {
method fmt (line 26) | fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
method hash (line 32) | fn hash<H: Hasher>(&self, state: &mut H) {
method as_any (line 40) | fn as_any(&self) -> &dyn Any {
method inputs (line 44) | fn inputs(&self) -> Vec<&LogicalPlan> {
method schema (line 48) | fn schema(&self) -> &DFSchemaRef {
method expressions (line 52) | fn expressions(&self) -> Vec<Expr> {
method fmt_for_explain (line 59) | fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result {
method from_template (line 63) | fn from_template(
method name (line 75) | fn name(&self) -> &str {
method dyn_hash (line 79) | fn dyn_hash(&self, state: &mut dyn Hasher) {
method dyn_eq (line 84) | fn dyn_eq(&self, other: &dyn UserDefinedLogicalNode) -> bool {
type PyPredictModel (line 93) | pub struct PyPredictModel {
method get_schema_name (line 100) | fn get_schema_name(&self) -> PyResult<Option<String>> {
method get_model_name (line 105) | fn get_model_name(&self) -> PyResult<String> {
method get_select (line 110) | fn get_select(&self) -> PyResult<PyLogicalPlan> {
type Error (line 116) | type Error = PyErr;
method try_from (line 118) | fn try_from(logical_plan: logical::LogicalPlan) -> Result<Self, Self::...
FILE: src/sql/logical/projection.rs
type PyProjection (line 13) | pub struct PyProjection {
method projected_expressions (line 19) | fn projected_expressions(&mut self, local_expr: &PyExpr) -> Vec<PyExpr> {
method named_projects (line 36) | fn named_projects(&mut self) -> PyResult<Vec<(String, PyExpr)>> {
type Error (line 60) | type Error = PyErr;
method try_from (line 62) | fn try_from(logical_plan: LogicalPlan) -> Result<Self, Self::Error> {
FILE: src/sql/logical/repartition_by.rs
type PyRepartitionBy (line 14) | pub struct PyRepartitionBy {
method get_select_query (line 21) | fn get_select_query(&self) -> PyResult<logical::PyLogicalPlan> {
method get_distribute_list (line 27) | fn get_distribute_list(&self) -> PyResult<Vec<PyExpr>> {
method get_distribute_columns (line 38) | fn get_distribute_columns(&self) -> PyResult<String> {
type Error (line 53) | type Error = PyErr;
method try_from (line 55) | fn try_from(logical_plan: LogicalPlan) -> Result<Self, Self::Error> {
FILE: src/sql/logical/show_columns.rs
type ShowColumnsPlanNode (line 22) | pub struct ShowColumnsPlanNode {
method fmt (line 29) | fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
method hash (line 35) | fn hash<H: Hasher>(&self, state: &mut H) {
method as_any (line 43) | fn as_any(&self) -> &dyn Any {
method inputs (line 47) | fn inputs(&self) -> Vec<&LogicalPlan> {
method schema (line 51) | fn schema(&self) -> &DFSchemaRef {
method expressions (line 55) | fn expressions(&self) -> Vec<Expr> {
method fmt_for_explain (line 62) | fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result {
method from_template (line 66) | fn from_template(
method name (line 78) | fn name(&self) -> &str {
method dyn_hash (line 82) | fn dyn_hash(&self, state: &mut dyn Hasher) {
method dyn_eq (line 87) | fn dyn_eq(&self, other: &dyn UserDefinedLogicalNode) -> bool {
type PyShowColumns (line 96) | pub struct PyShowColumns {
method get_table_name (line 103) | fn get_table_name(&self) -> PyResult<String> {
method get_schema_name (line 108) | fn get_schema_name(&self) -> PyResult<Option<String>> {
type Error (line 114) | type Error = PyErr;
method try_from (line 116) | fn try_from(logical_plan: logical::LogicalPlan) -> Result<Self, Self::...
FILE: src/sql/logical/show_models.rs
type ShowModelsPlanNode (line 18) | pub struct ShowModelsPlanNode {
method fmt (line 24) | fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
method hash (line 30) | fn hash<H: Hasher>(&self, state: &mut H) {
method as_any (line 37) | fn as_any(&self) -> &dyn Any {
method inputs (line 41) | fn inputs(&self) -> Vec<&LogicalPlan> {
method schema (line 45) | fn schema(&self) -> &DFSchemaRef {
method expressions (line 49) | fn expressions(&self) -> Vec<Expr> {
method fmt_for_explain (line 56) | fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result {
method from_template (line 60) | fn from_template(
method name (line 71) | fn name(&self) -> &str {
method dyn_hash (line 75) | fn dyn_hash(&self, state: &mut dyn Hasher) {
method dyn_eq (line 80) | fn dyn_eq(&self, other: &dyn UserDefinedLogicalNode) -> bool {
type PyShowModels (line 89) | pub struct PyShowModels {
method get_schema_name (line 96) | fn get_schema_name(&self) -> PyResult<Option<String>> {
type Error (line 102) | type Error = PyErr;
method try_from (line 104) | fn try_from(logical_plan: LogicalPlan) -> Result<Self, Self::Error> {
FILE: src/sql/logical/show_schemas.rs
type ShowSchemasPlanNode (line 22) | pub struct ShowSchemasPlanNode {
method fmt (line 29) | fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
method hash (line 35) | fn hash<H: Hasher>(&self, state: &mut H) {
method as_any (line 42) | fn as_any(&self) -> &dyn Any {
method inputs (line 46) | fn inputs(&self) -> Vec<&LogicalPlan> {
method schema (line 50) | fn schema(&self) -> &DFSchemaRef {
method expressions (line 54) | fn expressions(&self) -> Vec<Expr> {
method fmt_for_explain (line 61) | fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result {
method from_template (line 65) | fn from_template(
method name (line 77) | fn name(&self) -> &str {
method dyn_hash (line 81) | fn dyn_hash(&self, state: &mut dyn Hasher) {
method dyn_eq (line 86) | fn dyn_eq(&self, other: &dyn UserDefinedLogicalNode) -> bool {
type PyShowSchema (line 95) | pub struct PyShowSchema {
method get_from (line 102) | fn get_from(&self) -> PyResult<Option<String>> {
method get_like (line 107) | fn get_like(&self) -> PyResult<Option<String>> {
type Error (line 113) | type Error = PyErr;
method try_from (line 115) | fn try_from(logical_plan: logical::LogicalPlan) -> Result<Self, Self::...
FILE: src/sql/logical/show_tables.rs
type ShowTablesPlanNode (line 22) | pub struct ShowTablesPlanNode {
method fmt (line 29) | fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
method hash (line 35) | fn hash<H: Hasher>(&self, state: &mut H) {
method as_any (line 42) | fn as_any(&self) -> &dyn Any {
method inputs (line 46) | fn inputs(&self) -> Vec<&LogicalPlan> {
method schema (line 50) | fn schema(&self) -> &DFSchemaRef {
method expressions (line 54) | fn expressions(&self) -> Vec<Expr> {
method fmt_for_explain (line 61) | fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result {
method from_template (line 69) | fn from_template(
method name (line 81) | fn name(&self) -> &str {
method dyn_hash (line 85) | fn dyn_hash(&self, state: &mut dyn Hasher) {
method dyn_eq (line 90) | fn dyn_eq(&self, other: &dyn UserDefinedLogicalNode) -> bool {
type PyShowTables (line 99) | pub struct PyShowTables {
method get_catalog_name (line 106) | fn get_catalog_name(&self) -> PyResult<Option<String>> {
method get_schema_name (line 111) | fn get_schema_name(&self) -> PyResult<Option<String>> {
type Error (line 117) | type Error = PyErr;
method try_from (line 119) | fn try_from(logical_plan: logical::LogicalPlan) -> Result<Self, Self::...
FILE: src/sql/logical/sort.rs
type PySort (line 11) | pub struct PySort {
method sort_expressions (line 19) | pub fn sort_expressions(&self) -> PyResult<Vec<PyExpr>> {
method get_fetch_val (line 24) | pub fn get_fetch_val(&self) -> PyResult<Option<usize>> {
type Error (line 30) | type Error = PyErr;
method try_from (line 32) | fn try_from(logical_plan: LogicalPlan) -> Result<Self, Self::Error> {
FILE: src/sql/logical/subquery_alias.rs
type PySubqueryAlias (line 8) | pub struct PySubqueryAlias {
method alias (line 16) | pub fn alias(&self) -> PyResult<String> {
type Error (line 22) | type Error = PyErr;
method try_from (line 24) | fn try_from(logical_plan: LogicalPlan) -> Result<Self, Self::Error> {
FILE: src/sql/logical/table_scan.rs
type PyTableScan (line 22) | pub struct PyTableScan {
method _valid_expr_type (line 45) | fn _valid_expr_type(expr: &[Expr]) -> bool {
method _expand_dnf_filter (line 52) | pub fn _expand_dnf_filter(
method _expand_dnf_filters (line 160) | pub fn _expand_dnf_filters(
method scan_projects (line 187) | fn scan_projects(&mut self) -> PyResult<Vec<String>> {
method contains_projections (line 203) | fn contains_projections(&self) -> bool {
method scan_filters (line 208) | fn scan_filters(&self) -> PyResult<Vec<PyExpr>> {
method dnf_io_filters (line 213) | fn dnf_io_filters(&self, py: Python) -> PyResult<PyFilteredResult> {
type Error (line 220) | type Error = PyErr;
method try_from (line 222) | fn try_from(logical_plan: LogicalPlan) -> Result<Self, Self::Error> {
type FilterTuple (line 27) | type FilterTuple = (String, String, Option<Vec<PyObject>>);
type PyFilteredResult (line 30) | pub struct PyFilteredResult {
FILE: src/sql/logical/use_schema.rs
type UseSchemaPlanNode (line 18) | pub struct UseSchemaPlanNode {
method fmt (line 24) | fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
method hash (line 30) | fn hash<H: Hasher>(&self, state: &mut H) {
method as_any (line 37) | fn as_any(&self) -> &dyn Any {
method inputs (line 41) | fn inputs(&self) -> Vec<&LogicalPlan> {
method schema (line 45) | fn schema(&self) -> &DFSchemaRef {
method expressions (line 49) | fn expressions(&self) -> Vec<Expr> {
method fmt_for_explain (line 56) | fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result {
method from_template (line 60) | fn from_template(
method name (line 71) | fn name(&self) -> &str {
method dyn_hash (line 75) | fn dyn_hash(&self, state: &mut dyn Hasher) {
method dyn_eq (line 80) | fn dyn_eq(&self, other: &dyn UserDefinedLogicalNode) -> bool {
type PyUseSchema (line 89) | pub struct PyUseSchema {
method get_schema_name (line 96) | fn get_schema_name(&self) -> PyResult<String> {
type Error (line 102) | type Error = PyErr;
method try_from (line 104) | fn try_from(logical_plan: logical::LogicalPlan) -> Result<Self, Self::...
FILE: src/sql/logical/window.rs
type PyWindow (line 22) | pub struct PyWindow {
type Error (line 39) | type Error = PyErr;
method try_from (line 41) | fn try_from(logical_plan: LogicalPlan) -> Result<Self, Self::Error> {
method get_window_expr (line 65) | pub fn get_window_expr(&self) -> PyResult<Vec<PyExpr>> {
method get_sort_exprs (line 71) | pub fn get_sort_exprs(&self, expr: PyExpr) -> PyResult<Vec<PyExpr>> {
method get_partition_exprs (line 82) | pub fn get_partition_exprs(&self, expr: PyExpr) -> PyResult<Vec<PyExpr...
method get_args (line 93) | pub fn get_args(&self, expr: PyExpr) -> PyResult<Vec<PyExpr>> {
method window_func_name (line 104) | pub fn window_func_name(&self, expr: PyExpr) -> PyResult<String> {
method get_window_frame (line 113) | pub fn get_window_frame(&self, expr: PyExpr) -> Option<PyWindowFrame> {
type PyWindowFrame (line 28) | pub struct PyWindowFrame {
method from (line 50) | fn from(window_frame: WindowFrame) -> Self {
method get_frame_units (line 133) | pub fn get_frame_units(&self) -> PyResult<String> {
method get_lower_bound (line 138) | pub fn get_lower_bound(&self) -> PyResult<PyWindowFrameBound> {
method get_upper_bound (line 143) | pub fn get_upper_bound(&self) -> PyResult<PyWindowFrameBound> {
type PyWindowFrameBound (line 34) | pub struct PyWindowFrameBound {
method from (line 56) | fn from(frame_bound: WindowFrameBound) -> Self {
method is_current_row (line 152) | pub fn is_current_row(&self) -> bool {
method is_preceding (line 158) | pub fn is_preceding(&self) -> bool {
method is_following (line 164) | pub fn is_following(&self) -> bool {
method get_offset (line 169) | pub fn get_offset(&self) -> PyResult<Option<u64>> {
method is_unbounded (line 196) | pub fn is_unbounded(&self) -> PyResult<bool> {
function not_window_function_err (line 121) | fn not_window_function_err(expr: Expr) -> PyErr {
FILE: src/sql/optimizer.rs
type DaskSqlOptimizer (line 38) | pub struct DaskSqlOptimizer {
method new (line 45) | pub fn new(
method dynamic_partition_pruner (line 107) | pub fn dynamic_partition_pruner(fact_dimension_ratio: Option<f64>) -> ...
method optimize (line 119) | pub(crate) fn optimize(&self, plan: LogicalPlan) -> Result<LogicalPlan...
method optimize_once (line 126) | pub(crate) fn optimize_once(&self, plan: LogicalPlan) -> Result<Logica...
method observe (line 132) | fn observe(optimized_plan: &LogicalPlan, optimization: &dyn OptimizerR...
function subquery_filter_with_cast (line 159) | fn subquery_filter_with_cast() -> Result<()> {
function test_sql (line 172) | fn test_sql(sql: &str) -> Result<LogicalPlan> {
type MySchemaProvider (line 188) | struct MySchemaProvider {
method new (line 193) | fn new() -> Self {
method options (line 201) | fn options(&self) -> &ConfigOptions {
method get_table_provider (line 205) | fn get_table_provider(
method get_function_meta (line 230) | fn get_function_meta(&self, _name: &str) -> Option<Arc<ScalarUDF>> {
method get_aggregate_meta (line 234) | fn get_aggregate_meta(&self, _name: &str) -> Option<Arc<AggregateUDF>> {
method get_variable_type (line 238) | fn get_variable_type(&self, _variable_names: &[String]) -> Option<DataTy...
method get_window_meta (line 242) | fn get_window_meta(
type MyTableSource (line 250) | struct MyTableSource {
method as_any (line 255) | fn as_any(&self) -> &dyn Any {
method schema (line 259) | fn schema(&self) -> SchemaRef {
FILE: src/sql/optimizer/decorrelate_where_exists.rs
type DecorrelateWhereExists (line 41) | pub struct DecorrelateWhereExists {}
method new (line 45) | pub fn new() -> Self {
method extract_subquery_exprs (line 57) | fn extract_subquery_exprs(
method try_optimize (line 85) | fn try_optimize(
method name (line 121) | fn name(&self) -> &str {
method apply_order (line 125) | fn apply_order(&self) -> Option<ApplyOrder> {
function optimize_exists (line 150) | fn optimize_exists(
function optimize_subquery (line 178) | fn optimize_subquery(subquery: &LogicalPlan) -> Result<Option<(Expr, Log...
type SubqueryInfo (line 219) | struct SubqueryInfo {
method new (line 225) | pub fn new(query: Subquery, negated: bool) -> Self {
FILE: src/sql/optimizer/decorrelate_where_in.rs
type DecorrelateWhereIn (line 45) | pub struct DecorrelateWhereIn {
method new (line 51) | pub fn new() -> Self {
method extract_subquery_exprs (line 63) | fn extract_subquery_exprs(
method try_optimize (line 96) | fn try_optimize(
method name (line 128) | fn name(&self) -> &str {
method apply_order (line 132) | fn apply_order(&self) -> Option<ApplyOrder> {
function optimize_where_in (line 153) | fn optimize_where_in(
function remove_duplicated_filter (line 223) | fn remove_duplicated_filter(filters: Vec<Expr>, in_predicate: Expr) -> V...
function try_from_plan (line 244) | fn try_from_plan(plan: &LogicalPlan) -> Result<&Projection> {
type SubqueryInfo (line 253) | struct SubqueryInfo {
method new (line 260) | pub fn new(query: Subquery, expr: Expr, negated: bool) -> Self {
FILE: src/sql/optimizer/dynamic_partition_pruning.rs
type DynamicPartitionPruning (line 39) | pub struct DynamicPartitionPruning {
method new (line 45) | pub fn new(fact_dimension_ratio: f64) -> Self {
method name (line 53) | fn name(&self) -> &str {
method try_optimize (line 57) | fn try_optimize(
type JoinInfo (line 178) | struct JoinInfo {
function gather_joins (line 188) | fn gather_joins(plan: &LogicalPlan) -> HashSet<JoinInfo> {
type TableInfo (line 250) | struct TableInfo {
function gather_tables (line 263) | fn gather_tables(plan: &LogicalPlan) -> HashMap<String, TableInfo> {
function check_table_overlaps (line 356) | fn check_table_overlaps(
function get_filepath (line 369) | fn get_filepath(plan: &LogicalPlan) -> Option<&String> {
function get_table_size (line 380) | fn get_table_size(plan: &LogicalPlan) -> Option<usize> {
function gather_aliases (line 394) | fn gather_aliases(plan: &LogicalPlan) -> HashMap<String, String> {
type FloatWrapper (line 459) | struct FloatWrapper(f64);
method hash (line 464) | fn hash<H: Hasher>(&self, state: &mut H) {
type RowValue (line 474) | enum RowValue {
function read_table (line 484) | fn read_table(
function get_physical_type (line 640) | fn get_physical_type(schema: &Type, field: String) -> Option<BasicType> {
function get_repetition (line 669) | fn get_repetition(schema: &Type, field: String) -> Option<String> {
function get_filtered_fields (line 697) | fn get_filtered_fields(
function push_filtered_fields (line 742) | fn push_filtered_fields(
function satisfies_string (line 770) | fn satisfies_string(string_value: Option<&String>, filter: Expr) -> bool {
function satisfies_int64 (line 787) | fn satisfies_int64(long_value: Option<i64>, filter: Expr) -> bool {
function satisfies_int32 (line 824) | fn satisfies_int32(long_value: Option<i32>, filter: Expr) -> bool {
function satisfies_float (line 857) | fn satisfies_float(long_value: Option<f64>, filter: Expr) -> bool {
type RowHashSet (line 890) | type RowHashSet = HashSet<RowValue>;
type RowOptionHashSet (line 891) | type RowOptionHashSet = Option<RowHashSet>;
type RowTuple (line 892) | type RowTuple = (RowOptionHashSet, RowOptionHashSet);
type RowVec (line 893) | type RowVec = Vec<RowTuple>;
function combine_sets (line 900) | fn combine_sets(
function add_to_existing_set (line 968) | fn add_to_existing_set(
function optimize_table_scans (line 1000) | fn optimize_table_scans(
function format_inlist_expr (line 1037) | fn format_inlist_expr(
function optimize_children (line 1082) | fn optimize_children(
FILE: src/sql/optimizer/join_reorder.rs
type JoinReorder (line 15) | pub struct JoinReorder {
method new (line 28) | pub fn new(
method name (line 45) | fn name(&self) -> &str {
method try_optimize (line 49) | fn try_optimize(
function optimize_join (line 74) | fn optimize_join(
type Relation (line 191) | struct Relation {
method new (line 200) | fn new(plan: LogicalPlan) -> Self {
method has_filter (line 212) | fn has_filter(&self) -> bool {
function has_filter (line 217) | fn has_filter(plan: &LogicalPlan) -> bool {
function is_supported_join (line 247) | fn is_supported_join(join: &Join) -> bool {
function extract_inner_joins (line 271) | fn extract_inner_joins(plan: &LogicalPlan) -> (Vec<Relation>, HashSet<(E...
function find_join (line 321) | fn find_join(plan: &LogicalPlan) -> Option<Join> {
function get_unfiltered_dimensions (line 339) | fn get_unfiltered_dimensions(dims: &[Relation]) -> Vec<Relation> {
function get_filtered_dimensions (line 343) | fn get_filtered_dimensions(dims: &[Relation]) -> Vec<Relation> {
function build_join_tree (line 347) | fn build_join_tree(
function get_table_size (line 384) | fn get_table_size(plan: &LogicalPlan) -> Option<usize> {
FILE: src/sql/optimizer/utils.rs
function optimize_children (line 48) | pub fn optimize_children(
function split_conjunction (line 70) | pub fn split_conjunction(expr: &Expr) -> Vec<&Expr> {
function split_conjunction_impl (line 74) | fn split_conjunction_impl<'a>(expr: &'a Expr, mut exprs: Vec<&'a Expr>) ...
function extract_join_filters (line 97) | pub(crate) fn extract_join_filters(maybe_filter: &LogicalPlan) -> Result...
function split_conjunction_owned (line 135) | pub fn split_conjunction_owned(expr: Expr) -> Vec<Expr> {
function split_binary_owned (line 162) | pub fn split_binary_owned(expr: Expr, op: Operator) -> Vec<Expr> {
function split_binary_owned_impl (line 167) | fn split_binary_owned_impl(expr: Expr, operator: Operator, mut exprs: Ve...
function split_binary (line 185) | pub fn split_binary(expr: &Expr, op: Operator) -> Vec<&Expr> {
function split_binary_impl (line 190) | fn split_binary_impl<'a>(
function conjunction (line 230) | pub fn conjunction(filters: impl IntoIterator<Item = Expr>) -> Option<Ex...
function disjunction (line 240) | pub fn disjunction(filters: impl IntoIterator<Item = Expr>) -> Option<Ex...
function add_filter (line 247) | pub fn add_filter(plan: LogicalPlan, predicates: &[&Expr]) -> Result<Log...
function find_join_exprs (line 272) | pub fn find_join_exprs(exprs: Vec<&Expr>) -> Result<(Vec<Expr>, Vec<Expr...
function only_or_err (line 299) | pub fn only_or_err<T>(slice: &[T]) -> Result<&T> {
function merge_schema (line 313) | pub fn merge_schema(inputs: Vec<&LogicalPlan>) -> DFSchema {
function collect_subquery_cols (line 327) | pub(crate) fn collect_subquery_cols(
function replace_qualified_name (line 344) | pub(crate) fn replace_qualified_name(
function log_plan (line 360) | pub fn log_plan(description: &str, plan: &LogicalPlan) {
function test_split_conjunction (line 378) | fn test_split_conjunction() {
function test_split_conjunction_two (line 385) | fn test_split_conjunction_two() {
function test_split_conjunction_alias (line 395) | fn test_split_conjunction_alias() {
function test_split_conjunction_or (line 405) | fn test_split_conjunction_or() {
function test_split_binary_owned (line 412) | fn test_split_binary_owned() {
function test_split_binary_owned_two (line 418) | fn test_split_binary_owned_two() {
function test_split_binary_owned_different_op (line 426) | fn test_split_binary_owned_different_op() {
function test_split_conjunction_owned (line 436) | fn test_split_conjunction_owned() {
function test_split_conjunction_owned_two (line 442) | fn test_split_conjunction_owned_two() {
function test_split_conjunction_owned_alias (line 450) | fn test_split_conjunction_owned_alias() {
function test_conjunction_empty (line 462) | fn test_conjunction_empty() {
function test_conjunction (line 467) | fn test_conjunction() {
function test_disjunction_empty (line 479) | fn test_disjunction_empty() {
function test_disjunction (line 484) | fn test_disjunction() {
function test_split_conjunction_owned_or (line 496) | fn test_split_conjunction_owned_or() {
function test_collect_expr (line 502) | fn test_collect_expr() -> Result<()> {
FILE: src/sql/parser_utils.rs
type DaskParserUtils (line 3) | pub struct DaskParserUtils;
method elements_from_object_name (line 7) | pub fn elements_from_object_name(
FILE: src/sql/preoptimizer.rs
function extract_columns_and_literals (line 19) | fn extract_columns_and_literals(expr: &Expr) -> Vec<(Vec<Expr>, Expr)> {
function find_data_type (line 69) | fn find_data_type(column: Column, fields: Vec<DFField>) -> Option<DataTy...
function replace_literals (line 86) | fn replace_literals(expr: Expr, replacements: HashMap<Expr, Expr>) -> Ex...
function replace_binary_exprs (line 111) | fn replace_binary_exprs(expr: Expr, replacements: HashMap<Expr, Expr>) -...
function datetime_coercion (line 133) | pub fn datetime_coercion(plan: &LogicalPlan) -> Option<LogicalPlan> {
function optimize_children (line 187) | fn optimize_children(existing_plan: LogicalPlan) -> Option<LogicalPlan> {
FILE: src/sql/schema.rs
type DaskSchema (line 11) | pub struct DaskSchema {
method new (line 21) | pub fn new(schema_name: &str) -> Self {
method add_table (line 29) | pub fn add_table(&mut self, table: table::DaskTable) {
method add_or_overload_function (line 33) | pub fn add_or_overload_function(
FILE: src/sql/statement.rs
type PyStatement (line 7) | pub struct PyStatement {
method from (line 18) | fn from(statement: DaskStatement) -> PyStatement {
method new (line 24) | pub fn new(statement: DaskStatement) -> Self {
method from (line 12) | fn from(statement: PyStatement) -> DaskStatement {
FILE: src/sql/table.rs
type DaskTableSource (line 28) | pub struct DaskTableSource {
method new (line 36) | pub fn new(
method statistics (line 49) | pub fn statistics(&self) -> Option<&DaskStatistics> {
method filepath (line 54) | pub fn filepath(&self) -> Option<&String> {
method as_any (line 62) | fn as_any(&self) -> &dyn Any {
method schema (line 66) | fn schema(&self) -> SchemaRef {
method supports_filter_pushdown (line 70) | fn supports_filter_pushdown(
function is_supported_push_down_expr (line 88) | fn is_supported_push_down_expr(_expr: &Expr) -> bool {
type DaskStatistics (line 95) | pub struct DaskStatistics {
method new (line 102) | pub fn new(row_count: f64) -> Self {
method get_row_count (line 107) | pub fn get_row_count(&self) -> f64 {
type DaskTable (line 114) | pub struct DaskTable {
method new (line 125) | pub fn new(
method add_column (line 143) | pub fn add_column(&mut self, column_name: &str, type_map: DaskTypeMap) {
method get_schema (line 148) | pub fn get_schema(&self) -> PyResult<Option<String>> {
method get_table_name (line 153) | pub fn get_table_name(&self) -> PyResult<String> {
method qualified_name (line 158) | pub fn qualified_name(&self, plan: logical::PyLogicalPlan) -> Vec<Stri...
method row_type (line 177) | pub fn row_type(&self) -> RelDataType {
function table_from_logical_plan (line 187) | pub(crate) fn table_from_logical_plan(
FILE: src/sql/types.rs
type RexType (line 16) | pub enum RexType {
type DaskTypeMap (line 35) | pub struct DaskTypeMap {
method from (line 42) | pub fn from(sql_type: SqlTypeName, data_type: PyDataType) -> Self {
method new (line 54) | fn new(sql_type: SqlTypeName, py_kwargs: Option<&PyDict>) -> PyResult<...
method __str__ (line 154) | fn __str__(&self) -> String {
method sql_type (line 159) | pub fn sql_type(&self) -> SqlTypeName {
method data_type (line 164) | pub fn data_type(&self) -> PyDataType {
type PyDataType (line 171) | pub struct PyDataType {
method get_precision_scale (line 179) | pub fn get_precision_scale(&self) -> PyResult<(u8, i8)> {
method from (line 201) | fn from(data_type: DataType) -> PyDataType {
method from (line 195) | fn from(data_type: PyDataType) -> DataType {
type SqlTypeName (line 214) | pub enum SqlTypeName {
method to_arrow (line 267) | pub fn to_arrow(&self) -> Result<DataType, DaskPlannerError> {
method from_arrow (line 286) | pub fn from_arrow(arrow_type: &DataType) -> Result<Self, DaskPlannerEr...
method py_from_string (line 332) | pub fn py_from_string(input_type: &str) -> PyResult<Self> {
method from_string (line 338) | pub fn from_string(input_type: &str) -> Result<Self, DaskPlannerError> {
function invalid_type_name (line 415) | fn invalid_type_name() {
function string (line 425) | fn string() {
function varchar_n (line 430) | fn varchar_n() {
function decimal_p_s (line 435) | fn decimal_p_s() {
function assert_expected (line 439) | fn assert_expected(expected: &str, input: &str) {
FILE: src/sql/types/rel_data_type.rs
constant PRECISION_NOT_SPECIFIED (line 7) | const PRECISION_NOT_SPECIFIED: i32 = i32::MIN;
constant SCALE_NOT_SPECIFIED (line 8) | const SCALE_NOT_SPECIFIED: i32 = -1;
type RelDataType (line 13) | pub struct RelDataType {
method new (line 22) | pub fn new(nullable: bool, fields: Vec<RelDataTypeField>) -> Self {
method field (line 36) | pub fn field(&self, field_name: &str, case_sensitive: bool) -> PyResul...
method field_map (line 62) | pub fn field_map(&self) -> HashMap<String, RelDataTypeField> {
method field_list (line 72) | pub fn field_list(&self) -> Vec<RelDataTypeField> {
method field_names (line 78) | pub fn field_names(&self) -> Vec<String> {
method field_count (line 88) | pub fn field_count(&self) -> usize {
method is_struct (line 93) | pub fn is_struct(&self) -> bool {
method is_nullable (line 99) | pub fn is_nullable(&self) -> bool {
method precision (line 104) | pub fn precision(&self) -> i32 {
method scale (line 109) | pub fn scale(&self) -> i32 {
FILE: src/sql/types/rel_data_type_field.rs
type RelDataTypeField (line 17) | pub struct RelDataTypeField {
method from (line 26) | pub fn from(field: &DFField, schema: &DFSchema) -> Result<RelDataTypeF...
method new (line 45) | pub fn new(name: &str, type_map: DaskTypeMap, index: usize) -> Self {
method qualifier (line 55) | pub fn qualifier(&self) -> Option<String> {
method name (line 60) | pub fn name(&self) -> &str {
method qualified_name (line 65) | pub fn qualified_name(&self) -> String {
method index (line 73) | pub fn index(&self) -> usize {
method data_type (line 78) | pub fn data_type(&self) -> DaskTypeMap {
method get_key (line 86) | pub fn get_key(&self) -> &str {
method get_value (line 94) | pub fn get_value(&self) -> DaskTypeMap {
method set_value (line 99) | pub fn set_value(&mut self, data_type: DaskTypeMap) {
method fmt (line 111) | fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result {
FILE: tests/integration/fixtures.py
function df_simple (line 32) | def df_simple():
function df_wide (line 37) | def df_wide():
function df (line 50) | def df():
function department_table (line 61) | def department_table():
function user_table_1 (line 66) | def user_table_1():
function user_table_2 (line 71) | def user_table_2():
function long_table (line 76) | def long_table():
function user_table_inf (line 81) | def user_table_inf():
function user_table_nan (line 86) | def user_table_nan():
function string_table (line 91) | def string_table():
function datetime_table (line 105) | def datetime_table():
function timeseries (line 122) | def timeseries():
function parquet_ddf (line 127) | def parquet_ddf(tmpdir):
function gpu_user_table_1 (line 151) | def gpu_user_table_1(user_table_1):
function gpu_df (line 156) | def gpu_df(df):
function gpu_long_table (line 161) | def gpu_long_table(long_table):
function gpu_string_table (line 166) | def gpu_string_table(string_table):
function gpu_datetime_table (line 171) | def gpu_datetime_table(datetime_table):
function gpu_timeseries (line 190) | def gpu_timeseries(timeseries):
function c (line 195) | def c(
function temporary_data_file (line 256) | def temporary_data_file():
function assert_query_gives_same_result (line 268) | def assert_query_gives_same_result(engine):
function gpu_client (line 352) | def gpu_client(request):
function client (line 367) | def client():
FILE: tests/integration/test_analyze.py
function test_analyze (line 8) | def test_analyze(c, df):
FILE: tests/integration/test_cmd.py
function mock_prompt_input (line 15) | def mock_prompt_input():
function _feed_cli_with_input (line 30) | def _feed_cli_with_input(
function test_meta_commands (line 69) | def test_meta_commands(c, client, capsys):
function test_connection_info (line 120) | def test_connection_info(c, client, capsys):
function test_quit (line 130) | def test_quit(c, client, capsys):
function test_non_meta_commands (line 138) | def test_non_meta_commands(c, client, capsys):
FILE: tests/integration/test_compatibility.py
function eq_sqlite (line 25) | def eq_sqlite(sql, **dfs):
function make_rand_df (line 50) | def make_rand_df(size: int, **kwargs):
function test_basic_select_from (line 86) | def test_basic_select_from():
function test_case_when (line 97) | def test_case_when():
function test_drop_duplicates (line 113) | def test_drop_duplicates():
function test_order_by_no_limit (line 143) | def test_order_by_no_limit():
function test_order_by_limit (line 154) | def test_order_by_limit():
function test_where (line 177) | def test_where():
function test_in_between (line 198) | def test_in_between():
function test_join_inner (line 210) | def test_join_inner():
function test_join_left (line 226) | def test_join_left():
function test_join_cross (line 241) | def test_join_cross():
function test_join_multi (line 255) | def test_join_multi():
function test_single_agg_count_no_group_by (line 272) | def test_single_agg_count_no_group_by():
function test_multi_agg_count_no_group_by (line 287) | def test_multi_agg_count_no_group_by():
function test_multi_agg_count_no_group_by_dupe_distinct (line 310) | def test_multi_agg_count_no_group_by_dupe_distinct():
function test_agg_count_distinct_group_by (line 334) | def test_agg_count_distinct_group_by():
function test_agg_count_no_group_by (line 351) | def test_agg_count_no_group_by():
function test_agg_count_distinct_no_group_by (line 365) | def test_agg_count_distinct_no_group_by():
function test_agg_count (line 379) | def test_agg_count():
function test_agg_sum_avg_no_group_by (line 402) | def test_agg_sum_avg_no_group_by():
function test_agg_sum_avg (line 432) | def test_agg_sum_avg():
function test_agg_min_max_no_group_by (line 454) | def test_agg_min_max_no_group_by():
function test_agg_min_max (line 490) | def test_agg_min_max():
function test_window_row_number (line 525) | def test_window_row_number():
function test_window_row_number_partition_by (line 556) | def test_window_row_number_partition_by():
function test_window_ranks (line 584) | def test_window_ranks():
function test_window_ranks_partition_by (line 601) | def test_window_ranks_partition_by():
function test_window_lead_lag (line 621) | def test_window_lead_lag():
function test_window_lead_lag_partition_by (line 644) | def test_window_lead_lag_partition_by():
function test_window_sum_avg (line 660) | def test_window_sum_avg():
function test_window_sum_avg_partition_by (line 698) | def test_window_sum_avg_partition_by():
function test_window_min_max (line 734) | def test_window_min_max():
function test_window_min_max_partition_by (line 783) | def test_window_min_max_partition_by():
function test_window_count (line 818) | def test_window_count():
function test_window_count_partition_by (line 870) | def test_window_count_partition_by():
function test_nested_query (line 914) | def test_nested_query():
function test_union (line 929) | def test_union():
function test_except (line 960) | def test_except():
function test_intersect (line 979) | def test_intersect():
function test_with (line 995) | def test_with():
function test_integration_1 (line 1015) | def test_integration_1():
function test_query_case_sensitivity (line 1050) | def test_query_case_sensitivity(case_sensitive):
function test_column_name_starting_with_number (line 1078) | def test_column_name_starting_with_number():
FILE: tests/integration/test_complex.py
function test_complex_query (line 4) | def test_complex_query(c):
FILE: tests/integration/test_create.py
function test_create_from_csv (line 10) | def test_create_from_csv(c, df, temporary_data_file, gpu):
function test_cluster_memory (line 41) | def test_cluster_memory(client, c, df, gpu):
function test_create_from_csv_persist (line 68) | def test_create_from_csv_persist(c, df, temporary_data_file, gpu):
function test_wrong_create (line 93) | def test_wrong_create(c):
function test_create_from_query (line 118) | def test_create_from_query(c, df):
function test_view_table_persist (line 188) | def test_view_table_persist(c, temporary_data_file, df, gpu):
function test_replace_and_error (line 238) | def test_replace_and_error(c, temporary_data_file, df):
function test_drop (line 365) | def test_drop(c):
function test_create_gpu_error (line 390) | def test_create_gpu_error(c, df, temporary_data_file):
FILE: tests/integration/test_distributeby.py
function test_distribute_by (line 7) | def test_distribute_by(c, gpu):
FILE: tests/integration/test_explain.py
function test_sql_query_explain (line 9) | def test_sql_query_explain(c, gpu):
function test_statistics_explain (line 28) | def test_statistics_explain(c, gpu):
FILE: tests/integration/test_filter.py
function test_filter (line 13) | def test_filter(c, df):
function test_filter_scalar (line 20) | def test_filter_scalar(c, df):
function test_filter_complicated (line 42) | def test_filter_complicated(c, df):
function test_filter_with_nan (line 52) | def test_filter_with_nan(c):
function test_string_filter (line 62) | def test_string_filter(c, string_table):
function test_filter_cast_date (line 86) | def test_filter_cast_date(c, input_table, request):
function test_filter_cast_timestamp (line 114) | def test_filter_cast_timestamp(c, input_table, request):
function test_filter_year (line 130) | def test_filter_year(c):
function test_predicate_pushdown (line 212) | def test_predicate_pushdown(c, parquet_ddf, query, df_func, filters):
function test_filtered_csv (line 238) | def test_filtered_csv(tmpdir, c):
function test_filter_decimal (line 270) | def test_filter_decimal(c, gpu):
function test_predicate_pushdown_isna (line 317) | def test_predicate_pushdown_isna(tmpdir):
FILE: tests/integration/test_fugue.py
function test_fugue_workflow (line 13) | def test_fugue_workflow(client):
function test_fugue_fsql (line 27) | def test_fugue_fsql(client):
function test_dask_fsql (line 44) | def test_dask_fsql(client):
FILE: tests/integration/test_function.py
function test_custom_function (line 13) | def test_custom_function(c, df):
function test_custom_function_row (line 24) | def test_custom_function_row(c, df):
function test_custom_function_any_colnames (line 36) | def test_custom_function_any_colnames(colnames, df_wide, c):
function test_custom_function_row_return_types (line 59) | def test_custom_function_row_return_types(c, df, retty):
function test_custom_function_row_args (line 76) | def test_custom_function_row_args(c, df, k, op, retty):
function test_custom_function_row_two_args (line 102) | def test_custom_function_row_two_args(c, df, k1, k2, op, retty):
function test_multiple_definitions (line 132) | def test_multiple_definitions(c, df_simple):
function test_aggregate_function (line 166) | def test_aggregate_function(c):
function test_reregistration (line 180) | def test_reregistration(c):
function test_unsupported_dtype (line 211) | def test_unsupported_dtype(c, dtype):
function test_wrong_input_type (line 225) | def test_wrong_input_type(c):
FILE: tests/integration/test_groupby.py
function timeseries_df (line 11) | def timeseries_df(c):
function test_group_by (line 25) | def test_group_by(c):
function test_group_by_multi (line 40) | def test_group_by_multi(c, gpu):
function test_group_by_all (line 70) | def test_group_by_all(c, df):
function test_group_by_filtered (line 110) | def test_group_by_filtered(c):
function test_group_by_case (line 155) | def test_group_by_case(c):
function test_group_by_nan (line 174) | def test_group_by_nan(c, user_table_nan):
function test_aggregations (line 205) | def test_aggregations(c):
function test_stddev (line 291) | def test_stddev(c, gpu):
function test_regr_aggregation (line 364) | def test_regr_aggregation(c, timeseries_df, gpu):
function test_covar_aggregation (line 428) | def test_covar_aggregation(c, timeseries_df):
function test_groupby_split_out (line 492) | def test_groupby_split_out(c, input_table, split_out, request):
function test_groupby_split_every (line 527) | def test_groupby_split_every(c, gpu):
function test_agg_decimal (line 602) | def test_agg_decimal(c, gpu):
FILE: tests/integration/test_hive.py
function hive_cursor (line 40) | def hive_cursor():
function test_select (line 214) | def test_select(hive_cursor):
function test_select_partitions (line 224) | def test_select_partitions(hive_cursor):
function test_select_multipartitions (line 235) | def test_select_multipartitions(hive_cursor):
FILE: tests/integration/test_intake.py
function intake_catalog_location (line 21) | def intake_catalog_location():
function check_read_table (line 47) | def check_read_table(c):
function test_intake_catalog (line 54) | def test_intake_catalog(intake_catalog_location):
function test_intake_location (line 62) | def test_intake_location(intake_catalog_location):
function test_intake_sql (line 71) | def test_intake_sql(intake_catalog_location):
FILE: tests/integration/test_jdbc.py
function c (line 19) | def c():
function app_client (line 32) | def app_client(c):
function test_jdbc_has_schema (line 46) | def test_jdbc_has_schema(app_client, c):
function test_jdbc_has_table (line 77) | def test_jdbc_has_table(app_client, c):
function test_jdbc_has_columns (line 95) | def test_jdbc_has_columns(app_client, c):
function assert_result (line 117) | def assert_result(result, col_len, data_len):
function create_table_row (line 125) | def create_table_row(a_str: str = "any", an_int: int = 1, a_float: float...
function check_data (line 133) | def check_data(app_client):
function get_result_or_error (line 142) | def get_result_or_error(app_client, response):
FILE: tests/integration/test_join.py
function test_join (line 14) | def test_join(c):
function test_join_inner (line 30) | def test_join_inner(c):
function test_join_outer (line 46) | def test_join_outer(c):
function test_join_left (line 68) | def test_join_left(c):
function test_join_left_anti (line 91) | def test_join_left_anti(c, gpu):
function test_join_left_semi (line 116) | def test_join_left_semi(c):
function test_join_right (line 140) | def test_join_right(c):
function test_join_cross (line 162) | def test_join_cross(c, user_table_1, department_table):
function test_join_complex (line 178) | def test_join_complex(c):
function test_join_literal (line 227) | def test_join_literal(c):
function test_conditional_join (line 260) | def test_conditional_join(c):
function test_join_on_unary_cond_only (line 284) | def test_join_on_unary_cond_only(c):
function test_join_case_projection_subquery (line 302) | def test_join_case_projection_subquery():
function test_conditional_join_with_limit (line 340) | def test_conditional_join_with_limit(c):
function test_intersect (line 371) | def test_intersect(c):
function test_intersect_multi_col (line 409) | def test_intersect_multi_col(c):
function test_join_alias_w_projection (line 433) | def test_join_alias_w_projection(c, parquet_ddf):
function test_filter_columns_post_join (line 442) | def test_filter_columns_post_join(c):
function test_join_reorder (line 460) | def test_join_reorder(c):
function check_broadcast_join (line 531) | def check_broadcast_join(df, val, raises=False):
function test_broadcast_join (line 556) | def test_broadcast_join(c, client, gpu):
function test_null_key_join (line 610) | def test_null_key_join(c):
FILE: tests/integration/test_model.py
function check_trained_model (line 26) | def check_trained_model(c, model_name="my_model", df_name="timeseries"):
function test_training_and_prediction (line 46) | def test_training_and_prediction(c, gpu_client):
function test_xgboost_training_prediction (line 112) | def test_xgboost_training_prediction(c, gpu_client):
function test_clustering_and_prediction (line 182) | def test_clustering_and_prediction(c, gpu_client):
function test_create_model_with_prediction (line 200) | def test_create_model_with_prediction(c):
function test_iterative_and_prediction (line 233) | def test_iterative_and_prediction(c):
function test_show_models (line 251) | def test_show_models(c):
function test_wrong_training_or_prediction (line 299) | def test_wrong_training_or_prediction(c):
function test_correct_argument_passing (line 338) | def test_correct_argument_passing(c):
function test_replace_and_error (line 381) | def test_replace_and_error(c):
function test_drop_model (line 460) | def test_drop_model(c):
function test_describe_model (line 484) | def test_describe_model(c):
function test_export_model (line 521) | def test_export_model(c, tmpdir):
function test_mlflow_export (line 586) | def test_mlflow_export(c, tmpdir):
function test_mlflow_export_xgboost (line 648) | def test_mlflow_export_xgboost(c, client, tmpdir):
function test_mlflow_export_lightgbm (line 682) | def test_mlflow_export_lightgbm(c, tmpdir):
function test_ml_experiment (line 716) | def test_ml_experiment(c, client):
function test_experiment_automl_classifier (line 921) | def test_experiment_automl_classifier(c, client):
function test_experiment_automl_regressor (line 948) | def test_experiment_automl_regressor(c, client):
function test_predict_with_nullable_types (line 978) | def test_predict_with_nullable_types(c):
function test_predict_with_limit_offset (line 1052) | def test_predict_with_limit_offset(c):
FILE: tests/integration/test_over.py
function test_over_with_sorting (line 7) | def test_over_with_sorting(c, user_table_1):
function test_over_with_partitioning (line 23) | def test_over_with_partitioning(c, user_table_2):
function test_over_with_grouping_and_sort (line 40) | def test_over_with_grouping_and_sort(c, user_table_1):
function test_over_with_different (line 56) | def test_over_with_different(c, user_table_1):
function test_over_calls (line 81) | def test_over_calls(c, user_table_1):
function test_over_single_value (line 123) | def test_over_single_value(c, user_table_1):
function test_over_with_windows (line 146) | def test_over_with_windows(c):
FILE: tests/integration/test_postgres.py
function engine (line 14) | def engine():
function test_select (line 57) | def test_select(assert_query_gives_same_result):
function test_join (line 122) | def test_join(assert_query_gives_same_result):
function test_sort (line 135) | def test_sort(assert_query_gives_same_result):
function test_limit (line 155) | def test_limit(assert_query_gives_same_result):
function test_groupby (line 178) | def test_groupby(assert_query_gives_same_result):
function test_filter (line 192) | def test_filter(assert_query_gives_same_result):
function test_string_operations (line 224) | def test_string_operations(assert_query_gives_same_result):
function test_string_position (line 265) | def test_string_position(assert_query_gives_same_result):
function test_string_overlay (line 277) | def test_string_overlay(assert_query_gives_same_result):
function test_statistical_functions (line 290) | def test_statistical_functions(assert_query_gives_same_result):
FILE: tests/integration/test_rex.py
function test_year (line 11) | def test_year(c, datetime_table):
function test_case (line 21) | def test_case(c, df):
function test_intervals (line 57) | def test_intervals(c):
function test_literals (line 88) | def test_literals(c):
function test_date_interval_math (line 114) | def test_date_interval_math(c):
function test_literal_null (line 131) | def test_literal_null(c):
function test_random (line 143) | def test_random(c):
function test_not (line 190) | def test_not(c, input_table, request):
function test_operators (line 205) | def test_operators(c, df):
function test_like (line 255) | def test_like(c, input_table, gpu, request):
function test_null (line 363) | def test_null(c):
function test_coalesce (line 399) | def test_coalesce(c, gpu):
function test_boolean_operations (line 456) | def test_boolean_operations(c):
function test_math_operations (line 488) | def test_math_operations(c, df):
function test_integer_div (line 548) | def test_integer_div(c, df_simple):
function test_subqueries (line 571) | def test_subqueries(c, user_table_1, user_table_2):
function test_string_functions (line 591) | def test_string_functions(c, gpu):
function test_string_position (line 669) | def test_string_position(c, gpu):
function test_string_overlay (line 703) | def test_string_overlay(c, gpu):
function test_date_functions (line 737) | def test_date_functions(c):
function test_timestampdiff (line 847) | def test_timestampdiff(c):
function test_totimestamp (line 942) | def test_totimestamp(c, gpu):
function test_extract_date (line 1056) | def test_extract_date(c, gpu):
function test_scalar_timestamps (line 1113) | def test_scalar_timestamps(c, gpu):
function test_datetime_coercion (line 1227) | def test_datetime_coercion(c):
FILE: tests/integration/test_sample.py
function get_system_sample (line 7) | def get_system_sample(df, fraction, seed):
function test_sample (line 25) | def test_sample(c, df):
FILE: tests/integration/test_schema.py
function test_table_schema (line 10) | def test_table_schema(c, df):
function test_function (line 40) | def test_function(c):
function test_create_schema (line 72) | def test_create_schema(c):
function test_drop_schema (line 83) | def test_drop_schema(c):
FILE: tests/integration/test_select.py
function test_select (line 11) | def test_select(c, df):
function test_select_alias (line 17) | def test_select_alias(c, df):
function test_select_column (line 27) | def test_select_column(c, df):
function test_select_different_types (line 33) | def test_select_different_types(c):
function test_select_expr (line 56) | def test_select_expr(c, df):
function test_select_of_select (line 70) | def test_select_of_select(c, df):
function test_select_of_select_with_casing (line 89) | def test_select_of_select_with_casing(c, df):
function test_wrong_input (line 108) | def test_wrong_input(c):
function test_timezones (line 116) | def test_timezones(c, datetime_table):
function test_limit (line 137) | def test_limit(c, input_table, limit, offset, request):
function test_date_casting (line 155) | def test_date_casting(c, input_table, request):
function test_timestamp_casting (line 188) | def test_timestamp_casting(c, input_table, request):
function test_multi_case_when (line 207) | def test_multi_case_when(c):
function test_case_when_no_else (line 224) | def test_case_when_no_else(c):
function test_singular_column_selection (line 241) | def test_singular_column_selection(c):
function test_multiple_column_projection (line 262) | def test_multiple_column_projection(c, parquet_ddf, input_cols):
function test_wildcard_select (line 277) | def test_wildcard_select(c):
FILE: tests/integration/test_server.py
function app_client (line 15) | def app_client():
function get_result_or_error (line 30) | def get_result_or_error(app_client, response):
function test_routes (line 59) | def test_routes(app_client):
function test_sql_query_cancel (line 68) | def test_sql_query_cancel(app_client):
function test_sql_query (line 81) | def test_sql_query(app_client):
function test_wrong_sql_query (line 102) | def test_wrong_sql_query(app_client):
function test_add_and_query (line 120) | def test_add_and_query(app_client, df, temporary_data_file):
function test_register_and_query (line 162) | def test_register_and_query(app_client, df):
function test_inf_table (line 190) | def test_inf_table(app_client, user_table_inf):
function test_nullable_int_table (line 213) | def test_nullable_int_table(app_client):
FILE: tests/integration/test_show.py
function test_schemas (line 9) | def test_schemas(c):
function test_tables (line 28) | def test_tables(gpu):
function test_columns (line 44) | def test_columns(c):
function test_wrong_input (line 65) | def test_wrong_input(c):
function test_show_tables (line 80) | def test_show_tables(c):
FILE: tests/integration/test_sort.py
function test_sort (line 16) | def test_sort(c, input_table_1, input_df, request):
function test_sort_by_alias (line 73) | def test_sort_by_alias(c, input_table_1, request):
function test_sort_with_nan (line 92) | def test_sort_with_nan(gpu):
function test_sort_with_nan_more_columns (line 182) | def test_sort_with_nan_more_columns(gpu):
function test_sort_with_nan_many_partitions (line 240) | def test_sort_with_nan_many_partitions(gpu):
function test_sort_strings (line 280) | def test_sort_strings(c, gpu):
function test_sort_not_allowed (line 299) | def test_sort_not_allowed(c, gpu):
function test_sort_by_old_alias (line 311) | def test_sort_by_old_alias(c, input_table_1, request):
function check_sort_topk (line 356) | def check_sort_topk(df, layer, contains=True):
function test_sort_topk (line 379) | def test_sort_topk(gpu):
FILE: tests/integration/test_sqlite.py
function engine (line 7) | def engine():
function test_select (line 11) | def test_select(assert_query_gives_same_result):
function test_join (line 43) | def test_join(assert_query_gives_same_result):
function test_sort (line 56) | def test_sort(assert_query_gives_same_result):
function test_limit (line 77) | def test_limit(assert_query_gives_same_result):
function test_groupby (line 101) | def test_groupby(assert_query_gives_same_result):
function test_calc (line 122) | def test_calc(assert_query_gives_same_result):
function test_filter (line 136) | def test_filter(assert_query_gives_same_result):
FILE: tests/integration/test_union.py
function test_union_not_all (line 6) | def test_union_not_all(c, df):
function test_union_all (line 20) | def test_union_all(c, df):
function test_union_mixed (line 35) | def test_union_mixed(c, df, long_table):
FILE: tests/unit/test_call.py
function test_operation (line 20) | def test_operation():
function test_reduce (line 30) | def test_reduce():
function test_case (line 36) | def test_case():
function test_is_true (line 51) | def test_is_true():
function test_is_false (line 74) | def test_is_false():
function test_like (line 97) | def test_like():
function test_not (line 109) | def test_not():
function test_nan (line 118) | def test_nan():
function test_simple_ops (line 130) | def test_simple_ops():
function test_math_operations (line 156) | def test_math_operations():
function test_string_operations (line 178) | def test_string_operations():
function test_dates (line 202) | def test_dates():
FILE: tests/unit/test_config.py
function test_custom_yaml (line 17) | def test_custom_yaml(tmpdir):
function test_env_variable (line 33) | def test_env_variable():
function test_default_config (line 40) | def test_default_config():
function test_schema (line 48) | def test_schema():
function test_schema_is_complete (line 65) | def test_schema_is_complete():
function test_dask_setconfig (line 98) | def test_dask_setconfig():
function test_dynamic_partition_pruning (line 111) | def test_dynamic_partition_pruning(tmpdir):
function test_dpp_single_file_parquet (line 161) | def test_dpp_single_file_parquet(tmpdir):
FILE: tests/unit/test_context.py
function test_add_remove_tables (line 23) | def test_add_remove_tables(gpu):
function test_sql (line 51) | def test_sql(gpu):
function test_input_types (line 82) | def test_input_types(temporary_data_file, gpu):
function test_tables_from_stack (line 125) | def test_tables_from_stack(gpu):
function test_function_adding (line 151) | def test_function_adding():
function test_aggregation_adding (line 220) | def test_aggregation_adding():
function test_alter_schema (line 289) | def test_alter_schema(c):
function test_alter_table (line 301) | def test_alter_table(c, df_simple):
function test_filepath (line 315) | def test_filepath(tmpdir, parquet_ddf):
function test_ddf_filepath (line 333) | def test_ddf_filepath(tmpdir, parquet_ddf):
FILE: tests/unit/test_datacontainer.py
function test_cc_init (line 4) | def test_cc_init():
function test_cc_limit_to (line 16) | def test_cc_limit_to():
function test_cc_rename (line 27) | def test_cc_rename():
function test_cc_add (line 38) | def test_cc_add():
FILE: tests/unit/test_mapping.py
function test_python_to_sql (line 11) | def test_python_to_sql():
function test_python_decimal_to_sql (line 21) | def test_python_decimal_to_sql():
function test_sql_to_python (line 29) | def test_sql_to_python():
function test_python_to_sql_to_python (line 36) | def test_python_to_sql_to_python():
function test_similar_type (line 45) | def test_similar_type():
FILE: tests/unit/test_ml_utils.py
function test_ml_class_mappings (line 22) | def test_ml_class_mappings(gpu):
function _check_axis_partitioning (line 42) | def _check_axis_partitioning(chunks, n_features):
function check_random_state (line 54) | def check_random_state(random_state):
function make_classification (line 67) | def make_classification(
function _assert_eq (line 99) | def _assert_eq(l, r, name=None, **kwargs):
function assert_estimator_equal (line 117) | def assert_estimator_equal(left, right, exclude=None, **kwargs):
function test_parallelpostfit_basic (line 147) | def test_parallelpostfit_basic():
function test_predict (line 163) | def test_predict(kind):
function test_transform (line 196) | def test_transform(kind):
function test_incremental_basic (line 219) | def test_incremental_basic(dataframes):
FILE: tests/unit/test_queries.py
function c (line 49) | def c(data_dir):
function gpu_c (line 68) | def gpu_c(data_dir):
function test_query (line 90) | def test_query(c, client, query, queries_dir):
function test_gpu_query (line 103) | def test_gpu_query(gpu_c, gpu_client, query, queries_dir):
FILE: tests/unit/test_statistics.py
function test_parquet_statistics (line 17) | def test_parquet_statistics(parquet_ddf, parallel):
function test_parquet_statistics_bad_args (line 40) | def test_parquet_statistics_bad_args(parquet_ddf):
function test_dc_statistics (line 62) | def test_dc_statistics(parquet_ddf):
FILE: tests/unit/test_utils.py
function test_is_frame_for_frame (line 11) | def test_is_frame_for_frame():
function test_is_frame_for_none (line 16) | def test_is_frame_for_none():
function test_is_frame_for_number (line 20) | def test_is_frame_for_number():
class PluginTest1 (line 25) | class PluginTest1(Pluggable):
class PluginTest2 (line 29) | class PluginTest2(Pluggable):
function test_add_plugin (line 33) | def test_add_plugin():
function test_overwrite (line 43) | def test_overwrite():
function test_predicate_pushdown_simple (line 61) | def test_predicate_pushdown_simple(parquet_ddf):
function test_predicate_pushdown_logical (line 74) | def test_predicate_pushdown_logical(parquet_ddf):
function test_predicate_pushdown_in (line 90) | def test_predicate_pushdown_in(parquet_ddf):
function test_predicate_pushdown_isna (line 111) | def test_predicate_pushdown_isna(parquet_ddf):
function test_predicate_pushdown_add_filters (line 132) | def test_predicate_pushdown_add_filters(parquet_ddf):
function test_predicate_pushdown_add_filters_no_extract (line 151) | def test_predicate_pushdown_add_filters_no_extract(parquet_ddf):
function test_predicate_pushdown_add_filters_no_preserve (line 168) | def test_predicate_pushdown_add_filters_no_preserve(parquet_ddf):
FILE: tests/utils.py
function assert_eq (line 15) | def assert_eq(*args, **kwargs):
function convert_nullable_columns (line 21) | def convert_nullable_columns(df):
function skipif_dask_expr_enabled (line 40) | def skipif_dask_expr_enabled(reason=None):
Condensed preview — 251 files, each showing path, character count, and a content snippet. Download the .json file or copy for the full structured content (1,389K chars).
[
{
"path": ".cargo/config.toml",
"chars": 224,
"preview": "[target.x86_64-apple-darwin]\nrustflags = [\n \"-C\", \"link-arg=-undefined\",\n \"-C\", \"link-arg=dynamic_lookup\",\n]\n\n[target."
},
{
"path": ".coveragerc",
"chars": 453,
"preview": "[run]\nomit = tests/*\nbranch = True\n\n[report]\n# Regexes for lines to exclude from consideration\nexclude_lines =\n # Hav"
},
{
"path": ".dockerignore",
"chars": 19,
"preview": "node_modules\n.next\n"
},
{
"path": ".github/CODEOWNERS",
"chars": 336,
"preview": "# global codeowners\n* @ayushdg @charlesbluca @galipremsagar\n\n# rust codeowners\n.cargo/ @ayushdg @cha"
},
{
"path": ".github/ISSUE_TEMPLATE/bug_report.md",
"chars": 906,
"preview": "---\nname: Bug report\nabout: Create a bug report to help us improve dask-sql\ntitle: \"[BUG]\"\nlabels: \"bug, needs triage\"\na"
},
{
"path": ".github/ISSUE_TEMPLATE/documentation-request.md",
"chars": 923,
"preview": "---\nname: Documentation request\nabout: Report incorrect or needed documentation\ntitle: \"[DOC]\"\nlabels: \"documentation\"\na"
},
{
"path": ".github/ISSUE_TEMPLATE/feature_request.md",
"chars": 671,
"preview": "---\nname: Feature request\nabout: Suggest an idea for dask-sql\ntitle: \"[ENH]\"\nlabels: \"enhancement, needs triage\"\nassigne"
},
{
"path": ".github/ISSUE_TEMPLATE/submit-question.md",
"chars": 152,
"preview": "---\nname: Submit question\nabout: Ask a general question about dask-sql\ntitle: \"[QST]\"\nlabels: \"question\"\nassignees: ''\n\n"
},
{
"path": ".github/dependabot.yml",
"chars": 711,
"preview": "version: 2\nupdates:\n - package-ecosystem: \"cargo\"\n directory: \"/\"\n schedule:\n interval: \"daily\"\n ignore:\n"
},
{
"path": ".github/workflows/conda.yml",
"chars": 4001,
"preview": "name: Build conda nightly\non:\n push:\n branches:\n - main\n pull_request:\n paths:\n - Cargo.toml\n - C"
},
{
"path": ".github/workflows/docker.yml",
"chars": 2650,
"preview": "name: Build Docker image\n\non:\n release:\n types: [created]\n push:\n branches:\n - main\n pull_request:\n pat"
},
{
"path": ".github/workflows/release.yml",
"chars": 4375,
"preview": "name: Upload Python package\non:\n release:\n types: [created]\n pull_request:\n paths:\n - .github/workflows/rel"
},
{
"path": ".github/workflows/rust.yml",
"chars": 2353,
"preview": "name: Test Rust package\n\non:\n # always trigger on PR\n push:\n branches:\n - main\n pull_request:\n # manual trig"
},
{
"path": ".github/workflows/style.yml",
"chars": 693,
"preview": "---\nname: Python style check\non: [pull_request]\n\n# When this workflow is queued, automatically cancel any previous runni"
},
{
"path": ".github/workflows/test-upstream.yml",
"chars": 5696,
"preview": "name: Nightly upstream testing\non:\n schedule:\n - cron: \"0 0 * * *\" # Daily “At 00:00” UTC\n workflow_dispatch: # all"
},
{
"path": ".github/workflows/test.yml",
"chars": 5098,
"preview": "name: Test Python package\non:\n push:\n branches:\n - main\n pull_request:\n\n# When this workflow is queued, automa"
},
{
"path": ".gitignore",
"chars": 700,
"preview": "# Byte-compiled / optimized / DLL files\n__pycache__/\n*.py[cod]\n*$py.class\n\n# Distribution / packaging\n.Python\nbuild/\ndev"
},
{
"path": ".pre-commit-config.yaml",
"chars": 1196,
"preview": "repos:\n - repo: https://github.com/psf/black\n rev: 22.10.0\n hooks:\n - id: black\n language_version: py"
},
{
"path": ".readthedocs.yaml",
"chars": 294,
"preview": "# See https://docs.readthedocs.io/en/stable/config-file/v2.html for details\nversion: 2\nbuild:\n os: ubuntu-20.04\n tools"
},
{
"path": "CODE_OF_CONDUCT.md",
"chars": 3358,
"preview": "# Contributor Covenant Code of Conduct\n\n## Our Pledge\n\nIn the interest of fostering an open and welcoming environment, w"
},
{
"path": "CONTRIBUTING.md",
"chars": 8693,
"preview": "# Contributing to Dask-SQL\n\n## Environment Setup\n\nThe environment used for development and CI consists of:\n\n- a system i"
},
{
"path": "Cargo.toml",
"chars": 773,
"preview": "[package]\nname = \"dask-sql\"\nrepository = \"https://github.com/dask-contrib/dask-sql\"\nversion = \"2024.5.0\"\ndescription = \""
},
{
"path": "LICENSE.txt",
"chars": 1067,
"preview": "MIT LICENCE\n\nCopyright (c) 2020 Nils Braun\n\nPermission is hereby granted, free of charge, to any person obtaining a copy"
},
{
"path": "MANIFEST.in",
"chars": 67,
"preview": "recursive-include dask_sql *.yaml\nrecursive-include dask_planner *\n"
},
{
"path": "README.md",
"chars": 8112,
"preview": "**Dask-SQL is currently not in active maintenance, see [#1344](https://github.com/dask-contrib/dask-sql/issues/1344) for"
},
{
"path": "conftest.py",
"chars": 1241,
"preview": "import dask\nimport pytest\n\npytest_plugins = [\"tests.integration.fixtures\"]\n\n\ndef pytest_addoption(parser):\n parser.ad"
},
{
"path": "continuous_integration/docker/cloud.dockerfile",
"chars": 318,
"preview": "ARG DOCKER_META_VERSION\nFROM nbraun/dask-sql:${DOCKER_META_VERSION}\n\nRUN conda config --add channels conda-forge \\\n &"
},
{
"path": "continuous_integration/docker/conda.txt",
"chars": 360,
"preview": "python>=3.9\ndask>=2024.4.1\npandas>=1.4.0\njpype1>=1.0.2\nopenjdk>=8\nmaven>=3.6.0\npytest>=6.0.2\npytest-cov>=2.10.1\npytest-x"
},
{
"path": "continuous_integration/docker/main.dockerfile",
"chars": 1556,
"preview": "# Dockerfile for dask-sql running the SQL server\n# For more information, see https://dask-sql.readthedocs.io/.\nFROM dask"
},
{
"path": "continuous_integration/environment-3.10.yaml",
"chars": 638,
"preview": "name: dask-sql\nchannels:\n- conda-forge\ndependencies:\n- c-compiler\n- dask>=2024.4.1\n- dask-expr>=1.0.11\n- docker-py>=7.1."
},
{
"path": "continuous_integration/environment-3.11.yaml",
"chars": 638,
"preview": "name: dask-sql\nchannels:\n- conda-forge\ndependencies:\n- c-compiler\n- dask>=2024.4.1\n- dask-expr>=1.0.11\n- docker-py>=7.1."
},
{
"path": "continuous_integration/environment-3.12.yaml",
"chars": 735,
"preview": "name: dask-sql\nchannels:\n- conda-forge\ndependencies:\n- c-compiler\n- dask>=2024.4.1\n- dask-expr>=1.0.11\n- docker-py>=7.1."
},
{
"path": "continuous_integration/environment-3.9.yaml",
"chars": 620,
"preview": "name: dask-sql-py39\nchannels:\n- conda-forge\ndependencies:\n- c-compiler\n- dask=2024.4.1\n- dask-expr=1.0.11\n- docker-py>=7"
},
{
"path": "continuous_integration/gpuci/environment-3.10.yaml",
"chars": 865,
"preview": "name: dask-sql\nchannels:\n- rapidsai\n- rapidsai-nightly\n- dask/label/dev\n- conda-forge\n- nvidia\n- nodefaults\ndependencies"
},
{
"path": "continuous_integration/gpuci/environment-3.11.yaml",
"chars": 865,
"preview": "name: dask-sql\nchannels:\n- rapidsai\n- rapidsai-nightly\n- dask/label/dev\n- conda-forge\n- nvidia\n- nodefaults\ndependencies"
},
{
"path": "continuous_integration/gpuci/environment-3.9.yaml",
"chars": 864,
"preview": "name: dask-sql\nchannels:\n- rapidsai\n- rapidsai-nightly\n- dask/label/dev\n- conda-forge\n- nvidia\n- nodefaults\ndependencies"
},
{
"path": "continuous_integration/recipe/build.sh",
"chars": 1928,
"preview": "#!/bin/bash\n\nset -ex\n\n# See https://github.com/conda-forge/rust-feedstock/blob/master/recipe/build.sh for cc env explana"
},
{
"path": "continuous_integration/recipe/conda_build_config.yaml",
"chars": 182,
"preview": "c_compiler:\n - gcc\nc_compiler_version:\n - '12'\nrust_compiler:\n - rust\nrust_compiler_version:\n - '1.72'\nmatur"
},
{
"path": "continuous_integration/recipe/meta.yaml",
"chars": 1479,
"preview": "{% set name = \"dask-sql\" %}\n{% set major_minor_patch = environ.get('GIT_DESCRIBE_TAG', '0.0.0.dev').split('.') %}\n{% set"
},
{
"path": "continuous_integration/recipe/run_test.py",
"chars": 683,
"preview": "import dask.dataframe as dd\nimport pandas as pd\n\nfrom dask_sql import Context\n\nc = Context()\n\ndata = \"\"\"\nname,x\nAlice,34"
},
{
"path": "continuous_integration/scripts/startup_script.py",
"chars": 76,
"preview": "from dask_sql.server.app import main\n\nif __name__ == \"__main__\":\n main()\n"
},
{
"path": "continuous_integration/scripts/update-dependencies.sh",
"chars": 369,
"preview": "#!/bin/bash\n\nUPDATE_ALL_CARGO_DEPS=\"${UPDATE_ALL_CARGO_DEPS:-true}\"\n# Update datafusion dependencies in the dask-planner"
},
{
"path": "dask_sql/__init__.py",
"chars": 627,
"preview": "# FIXME: can we modify TLS model of Rust object to avoid aarch64 glibc bug?\n# https://github.com/dask-contrib/dask-sql/i"
},
{
"path": "dask_sql/_compat.py",
"chars": 279,
"preview": "import prompt_toolkit\nfrom packaging.version import parse as parseVersion\n\n_prompt_toolkit_version = parseVersion(prompt"
},
{
"path": "dask_sql/cmd.py",
"chars": 9016,
"preview": "import logging\nimport os\nimport sys\nimport tempfile\nimport traceback\nfrom argparse import ArgumentParser\nfrom functools "
},
{
"path": "dask_sql/config.py",
"chars": 235,
"preview": "import os\n\nimport dask\nimport yaml\n\nfn = os.path.join(os.path.dirname(__file__), \"sql.yaml\")\n\nwith open(fn) as f:\n de"
},
{
"path": "dask_sql/context.py",
"chars": 38495,
"preview": "import asyncio\nimport inspect\nimport logging\nfrom collections import Counter\nfrom typing import Any, Callable, Union\n\nim"
},
{
"path": "dask_sql/datacontainer.py",
"chars": 9492,
"preview": "from collections import namedtuple\nfrom typing import Any, Union\n\nimport dask.dataframe as dd\nimport pandas as pd\n\nColum"
},
{
"path": "dask_sql/input_utils/__init__.py",
"chars": 493,
"preview": "from .convert import InputType, InputUtil\nfrom .dask import DaskInputPlugin\nfrom .hive import HiveInputPlugin\nfrom .inta"
},
{
"path": "dask_sql/input_utils/base.py",
"chars": 312,
"preview": "from typing import Any\n\n\nclass BaseInputPlugin:\n def is_correct_input(\n self, input_item: Any, table_name: str"
},
{
"path": "dask_sql/input_utils/convert.py",
"chars": 2671,
"preview": "import logging\nfrom typing import TYPE_CHECKING, Union\n\nimport dask.dataframe as dd\nimport pandas as pd\n\nfrom dask_sql.d"
},
{
"path": "dask_sql/input_utils/dask.py",
"chars": 910,
"preview": "from typing import Any\n\nimport dask.dataframe as dd\n\nfrom dask_sql.input_utils.base import BaseInputPlugin\n\n\nclass DaskI"
},
{
"path": "dask_sql/input_utils/hive.py",
"chars": 12080,
"preview": "import ast\nimport logging\nimport os\nfrom functools import partial\nfrom typing import Any, Union\n\nimport dask.dataframe a"
},
{
"path": "dask_sql/input_utils/intake.py",
"chars": 1080,
"preview": "from typing import Any\n\ntry:\n import intake\nexcept ImportError: # pragma: no cover\n intake = None\n\nfrom dask_sql."
},
{
"path": "dask_sql/input_utils/location.py",
"chars": 1773,
"preview": "import os\nfrom typing import Any\n\nimport dask.dataframe as dd\nfrom distributed.client import default_client\n\nfrom dask_s"
},
{
"path": "dask_sql/input_utils/pandaslike.py",
"chars": 1146,
"preview": "import dask.dataframe as dd\nimport pandas as pd\n\nfrom dask_sql.input_utils.base import BaseInputPlugin\n\n\nclass PandasLik"
},
{
"path": "dask_sql/input_utils/sqlalchemy.py",
"chars": 1222,
"preview": "from typing import Any\n\nfrom dask_sql.input_utils.hive import HiveInputPlugin\n\n\nclass SqlalchemyHiveInputPlugin(HiveInpu"
},
{
"path": "dask_sql/integrations/__init__.py",
"chars": 0,
"preview": ""
},
{
"path": "dask_sql/integrations/fugue.py",
"chars": 5097,
"preview": "try:\n import fugue\n import fugue_dask\n from dask.distributed import Client\n from fugue import WorkflowDataFr"
},
{
"path": "dask_sql/integrations/ipython.py",
"chars": 4577,
"preview": "import time\nfrom typing import TYPE_CHECKING\n\nfrom dask_sql.mappings import _SQL_TO_PYTHON_FRAMES\nfrom dask_sql.physical"
},
{
"path": "dask_sql/mappings.py",
"chars": 12822,
"preview": "import logging\nfrom datetime import datetime\nfrom typing import Any\n\nimport dask.array as da\nimport dask.config as dask_"
},
{
"path": "dask_sql/physical/__init__.py",
"chars": 0,
"preview": ""
},
{
"path": "dask_sql/physical/rel/__init__.py",
"chars": 34,
"preview": "from .convert import RelConverter\n"
},
{
"path": "dask_sql/physical/rel/base.py",
"chars": 4400,
"preview": "import logging\nfrom typing import TYPE_CHECKING, Optional\n\nimport dask.dataframe as dd\n\nfrom dask_sql.datacontainer impo"
},
{
"path": "dask_sql/physical/rel/convert.py",
"chars": 2271,
"preview": "import logging\nfrom typing import TYPE_CHECKING\n\nimport dask.dataframe as dd\n\nfrom dask_sql.physical.rel.base import Bas"
},
{
"path": "dask_sql/physical/rel/custom/__init__.py",
"chars": 1367,
"preview": "from .alter import AlterSchemaPlugin, AlterTablePlugin\nfrom .analyze_table import AnalyzeTablePlugin\nfrom .create_catalo"
},
{
"path": "dask_sql/physical/rel/custom/alter.py",
"chars": 2645,
"preview": "import logging\nfrom typing import TYPE_CHECKING\n\nfrom dask_sql.physical.rel.base import BaseRelPlugin\n\nlogger = logging."
},
{
"path": "dask_sql/physical/rel/custom/analyze_table.py",
"chars": 2304,
"preview": "from typing import TYPE_CHECKING\n\nimport dask.dataframe as dd\nimport pandas as pd\n\nfrom dask_sql.datacontainer import Co"
},
{
"path": "dask_sql/physical/rel/custom/create_catalog_schema.py",
"chars": 1180,
"preview": "import logging\nfrom typing import TYPE_CHECKING\n\nfrom dask_sql.physical.rel.base import BaseRelPlugin\n\nif TYPE_CHECKING:"
},
{
"path": "dask_sql/physical/rel/custom/create_experiment.py",
"chars": 9488,
"preview": "import logging\nfrom typing import TYPE_CHECKING\n\nimport dask.dataframe as dd\nimport pandas as pd\n\nfrom dask_sql.datacont"
},
{
"path": "dask_sql/physical/rel/custom/create_memory_table.py",
"chars": 2613,
"preview": "import logging\nfrom typing import TYPE_CHECKING\n\nfrom dask_sql.datacontainer import DataContainer\nfrom dask_sql.physical"
},
{
"path": "dask_sql/physical/rel/custom/create_model.py",
"chars": 8644,
"preview": "import logging\nimport warnings\nfrom typing import TYPE_CHECKING\n\nimport numpy as np\nfrom dask import delayed\n\nfrom dask_"
},
{
"path": "dask_sql/physical/rel/custom/create_table.py",
"chars": 2433,
"preview": "import logging\nfrom typing import TYPE_CHECKING\n\nfrom dask_sql.datacontainer import DataContainer\nfrom dask_sql.physical"
},
{
"path": "dask_sql/physical/rel/custom/describe_model.py",
"chars": 1425,
"preview": "from typing import TYPE_CHECKING\n\nimport dask.dataframe as dd\nimport pandas as pd\n\nfrom dask_sql.datacontainer import Co"
},
{
"path": "dask_sql/physical/rel/custom/distributeby.py",
"chars": 1336,
"preview": "import logging\nfrom typing import TYPE_CHECKING\n\nfrom dask_sql.datacontainer import ColumnContainer, DataContainer\nfrom "
},
{
"path": "dask_sql/physical/rel/custom/drop_model.py",
"chars": 1039,
"preview": "import logging\nfrom typing import TYPE_CHECKING\n\nfrom dask_sql.datacontainer import DataContainer\nfrom dask_sql.physical"
},
{
"path": "dask_sql/physical/rel/custom/drop_schema.py",
"chars": 887,
"preview": "import logging\nfrom typing import TYPE_CHECKING\n\nfrom dask_sql.physical.rel.base import BaseRelPlugin\n\nif TYPE_CHECKING:"
},
{
"path": "dask_sql/physical/rel/custom/drop_table.py",
"chars": 1448,
"preview": "import logging\nfrom typing import TYPE_CHECKING\n\nfrom dask_sql.datacontainer import DataContainer\nfrom dask_sql.physical"
},
{
"path": "dask_sql/physical/rel/custom/export_model.py",
"chars": 3950,
"preview": "import logging\nimport pickle\nfrom typing import TYPE_CHECKING\n\nfrom dask_sql.physical.rel.base import BaseRelPlugin\nfrom"
},
{
"path": "dask_sql/physical/rel/custom/metrics.py",
"chars": 6504,
"preview": "# Copyright 2017, Dask developers\n# Dask-ML project - https://github.com/dask/dask-ml\nfrom typing import Optional, TypeV"
},
{
"path": "dask_sql/physical/rel/custom/predict_model.py",
"chars": 3535,
"preview": "import logging\nimport uuid\nfrom typing import TYPE_CHECKING\n\nimport dask.dataframe as dd\nimport pandas as pd\n\nfrom dask_"
},
{
"path": "dask_sql/physical/rel/custom/show_columns.py",
"chars": 1470,
"preview": "from typing import TYPE_CHECKING\n\nimport dask.dataframe as dd\nimport pandas as pd\n\nfrom dask_sql.datacontainer import Co"
},
{
"path": "dask_sql/physical/rel/custom/show_models.py",
"chars": 920,
"preview": "from typing import TYPE_CHECKING\n\nimport dask.dataframe as dd\nimport pandas as pd\n\nfrom dask_sql.datacontainer import Co"
},
{
"path": "dask_sql/physical/rel/custom/show_schemas.py",
"chars": 1513,
"preview": "from typing import TYPE_CHECKING\n\nimport dask.dataframe as dd\nimport pandas as pd\n\nfrom dask_sql.datacontainer import Co"
},
{
"path": "dask_sql/physical/rel/custom/show_tables.py",
"chars": 1532,
"preview": "from typing import TYPE_CHECKING\n\nimport dask.dataframe as dd\nimport pandas as pd\n\nfrom dask_sql.datacontainer import Co"
},
{
"path": "dask_sql/physical/rel/custom/use_schema.py",
"chars": 909,
"preview": "from typing import TYPE_CHECKING\n\nfrom dask_sql.datacontainer import DataContainer\nfrom dask_sql.physical.rel.base impor"
},
{
"path": "dask_sql/physical/rel/custom/wrappers.py",
"chars": 28021,
"preview": "# Copyright 2017, Dask developers\n# Dask-ML project - https://github.com/dask/dask-ml\n\"\"\"Meta-estimators for parallelizi"
},
{
"path": "dask_sql/physical/rel/logical/__init__.py",
"chars": 916,
"preview": "from .aggregate import DaskAggregatePlugin\nfrom .cross_join import DaskCrossJoinPlugin\nfrom .empty import DaskEmptyRelat"
},
{
"path": "dask_sql/physical/rel/logical/aggregate.py",
"chars": 22811,
"preview": "import logging\nimport operator\nfrom collections import defaultdict\nfrom functools import reduce\nfrom typing import TYPE_"
},
{
"path": "dask_sql/physical/rel/logical/cross_join.py",
"chars": 1738,
"preview": "import logging\nfrom typing import TYPE_CHECKING\n\nimport dask_sql.utils as utils\nfrom dask_sql.datacontainer import Colum"
},
{
"path": "dask_sql/physical/rel/logical/empty.py",
"chars": 1116,
"preview": "import logging\nfrom typing import TYPE_CHECKING\n\nimport dask.dataframe as dd\nimport pandas as pd\n\nfrom dask_sql.datacont"
},
{
"path": "dask_sql/physical/rel/logical/explain.py",
"chars": 501,
"preview": "from typing import TYPE_CHECKING\n\nfrom dask_sql.physical.rel.base import BaseRelPlugin\n\nif TYPE_CHECKING:\n import das"
},
{
"path": "dask_sql/physical/rel/logical/filter.py",
"chars": 2326,
"preview": "import logging\nfrom typing import TYPE_CHECKING, List, Union\n\nimport dask.config as dask_config\nimport dask.dataframe as"
},
{
"path": "dask_sql/physical/rel/logical/join.py",
"chars": 12659,
"preview": "import logging\nimport operator\nimport warnings\nfrom functools import reduce\nfrom typing import TYPE_CHECKING\n\nimport das"
},
{
"path": "dask_sql/physical/rel/logical/limit.py",
"chars": 4418,
"preview": "from typing import TYPE_CHECKING\n\nimport dask.dataframe as dd\nfrom dask import config as dask_config\nfrom dask.blockwise"
},
{
"path": "dask_sql/physical/rel/logical/project.py",
"chars": 2595,
"preview": "import logging\nfrom typing import TYPE_CHECKING\n\nfrom dask_sql._datafusion_lib import RexType\nfrom dask_sql.datacontaine"
},
{
"path": "dask_sql/physical/rel/logical/sample.py",
"chars": 2114,
"preview": "import logging\nfrom typing import TYPE_CHECKING\n\nimport numpy as np\n\nfrom dask_sql.datacontainer import DataContainer\nfr"
},
{
"path": "dask_sql/physical/rel/logical/sort.py",
"chars": 1316,
"preview": "from typing import TYPE_CHECKING\n\nfrom dask_sql.datacontainer import DataContainer\nfrom dask_sql.physical.rel.base impor"
},
{
"path": "dask_sql/physical/rel/logical/subquery_alias.py",
"chars": 955,
"preview": "from typing import TYPE_CHECKING\n\nfrom dask_sql.datacontainer import DataContainer\nfrom dask_sql.physical.rel.base impor"
},
{
"path": "dask_sql/physical/rel/logical/table_scan.py",
"chars": 4213,
"preview": "import logging\nimport operator\nfrom functools import reduce\nfrom typing import TYPE_CHECKING\n\nfrom dask.dataframe import"
},
{
"path": "dask_sql/physical/rel/logical/union.py",
"chars": 1995,
"preview": "from typing import TYPE_CHECKING\n\nimport dask.dataframe as dd\n\nfrom dask_sql.datacontainer import ColumnContainer, DataC"
},
{
"path": "dask_sql/physical/rel/logical/values.py",
"chars": 2191,
"preview": "from typing import TYPE_CHECKING\n\nimport dask.dataframe as dd\nimport pandas as pd\n\nfrom dask_sql.datacontainer import Co"
},
{
"path": "dask_sql/physical/rel/logical/window.py",
"chars": 15173,
"preview": "import logging\nfrom collections import namedtuple\nfrom functools import partial\nfrom typing import TYPE_CHECKING, Callab"
},
{
"path": "dask_sql/physical/rex/__init__.py",
"chars": 34,
"preview": "from .convert import RexConverter\n"
},
{
"path": "dask_sql/physical/rex/base.py",
"chars": 812,
"preview": "import logging\nfrom typing import TYPE_CHECKING, Any, Union\n\nimport dask.dataframe as dd\n\nimport dask_sql\nfrom dask_sql."
},
{
"path": "dask_sql/physical/rex/convert.py",
"chars": 2417,
"preview": "import logging\nfrom typing import TYPE_CHECKING, Any, Union\n\nimport dask.dataframe as dd\n\nfrom dask_sql.datacontainer im"
},
{
"path": "dask_sql/physical/rex/core/__init__.py",
"chars": 319,
"preview": "from .alias import RexAliasPlugin\nfrom .call import RexCallPlugin\nfrom .input_ref import RexInputRefPlugin\nfrom .literal"
},
{
"path": "dask_sql/physical/rex/core/alias.py",
"chars": 1158,
"preview": "from typing import TYPE_CHECKING, Any, Union\n\nimport dask.dataframe as dd\n\nfrom dask_sql.datacontainer import DataContai"
},
{
"path": "dask_sql/physical/rex/core/call.py",
"chars": 39044,
"preview": "import logging\nimport operator\nimport re\nimport warnings\nfrom datetime import datetime\nfrom functools import partial, re"
},
{
"path": "dask_sql/physical/rex/core/input_ref.py",
"chars": 940,
"preview": "from typing import TYPE_CHECKING\n\nimport dask.dataframe as dd\n\nfrom dask_sql.datacontainer import DataContainer\nfrom das"
},
{
"path": "dask_sql/physical/rex/core/literal.py",
"chars": 7835,
"preview": "import logging\nfrom datetime import datetime\nfrom typing import TYPE_CHECKING, Any\n\nimport dask.dataframe as dd\nimport n"
},
{
"path": "dask_sql/physical/rex/core/subquery.py",
"chars": 993,
"preview": "from typing import TYPE_CHECKING\n\nimport dask.dataframe as dd\n\nfrom dask_sql.datacontainer import DataContainer\nfrom das"
},
{
"path": "dask_sql/physical/utils/__init__.py",
"chars": 0,
"preview": ""
},
{
"path": "dask_sql/physical/utils/filter.py",
"chars": 20460,
"preview": "from __future__ import annotations\n\nimport itertools\nimport logging\nimport operator\n\nimport dask.dataframe as dd\nimport "
},
{
"path": "dask_sql/physical/utils/groupby.py",
"chars": 1196,
"preview": "import dask.dataframe as dd\n\nfrom dask_sql.utils import new_temporary_column\n\n\ndef get_groupby_with_nulls_cols(\n df: "
},
{
"path": "dask_sql/physical/utils/ml_classes.py",
"chars": 6509,
"preview": "def get_cpu_classes():\n try:\n from sklearn.utils import all_estimators\n\n cpu_classes = {\n k:"
},
{
"path": "dask_sql/physical/utils/sort.py",
"chars": 4155,
"preview": "import dask.dataframe as dd\nimport pandas as pd\nfrom dask import config as dask_config\nfrom dask.utils import M\n\nfrom da"
},
{
"path": "dask_sql/physical/utils/statistics.py",
"chars": 7334,
"preview": "from __future__ import annotations\n\nimport itertools\nimport logging\nfrom collections import defaultdict\nfrom functools i"
},
{
"path": "dask_sql/server/__init__.py",
"chars": 0,
"preview": ""
},
{
"path": "dask_sql/server/app.py",
"chars": 8811,
"preview": "import asyncio\nimport logging\nfrom argparse import ArgumentParser\nfrom uuid import uuid4\n\nimport dask.distributed\nimport"
},
{
"path": "dask_sql/server/presto_jdbc.py",
"chars": 4983,
"preview": "import logging\n\nimport pandas as pd\n\nfrom dask_sql.context import Context\n\nlogger = logging.getLogger(__name__)\n\n\ndef cr"
},
{
"path": "dask_sql/server/responses.py",
"chars": 4195,
"preview": "import uuid\n\nimport dask.dataframe as dd\nimport numpy as np\nimport pandas as pd\nfrom fastapi import Request\n\nfrom dask_s"
},
{
"path": "dask_sql/sql-schema.yaml",
"chars": 4819,
"preview": "properties:\n\n sql:\n type: object\n properties:\n\n aggregate:\n type: object\n properties:\n\n "
},
{
"path": "dask_sql/sql.yaml",
"chars": 476,
"preview": "sql:\n aggregate:\n split_out: 1\n split_every: null\n\n identifier:\n case_sensitive: True\n\n join:\n broadcast:"
},
{
"path": "dask_sql/utils.py",
"chars": 5834,
"preview": "import importlib\nimport logging\nfrom collections import defaultdict\nfrom datetime import datetime\nfrom typing import Any"
},
{
"path": "docs/Makefile",
"chars": 638,
"preview": "# Minimal makefile for Sphinx documentation\n#\n\n# You can set these variables from the command line, and also\n# from the "
},
{
"path": "docs/environment.yml",
"chars": 434,
"preview": "name: dask-sql-docs\nchannels:\n - conda-forge\ndependencies:\n - python=3.9\n - sphinx>=4.0.0\n - sphinx-tabs\n - dask-sp"
},
{
"path": "docs/make.bat",
"chars": 804,
"preview": "@ECHO OFF\r\n\r\npushd %~dp0\r\n\r\nREM Command file for Sphinx documentation\r\n\r\nif \"%SPHINXBUILD%\" == \"\" (\r\n\tset SPHINXBUILD=sp"
},
{
"path": "docs/requirements-docs.txt",
"chars": 286,
"preview": "sphinx>=4.0.0\nsphinx-tabs\ndask-sphinx-theme>=3.0.0\ndask>=2024.4.1\npandas>=1.4.0\nfugue>=0.7.3\n# FIXME: https://github.com"
},
{
"path": "docs/source/api.rst",
"chars": 324,
"preview": ".. _api:\n\nAPI Documentation\n=================\n\n.. autoclass:: dask_sql.Context\n :members:\n :undoc-members:\n\n.. autof"
},
{
"path": "docs/source/best_practices.rst",
"chars": 9252,
"preview": ".. _best_practices:\n\nBest Practices and Performance Tips\n===================================\n\nSort and Use Read Filterin"
},
{
"path": "docs/source/cmd.rst",
"chars": 712,
"preview": ".. _cmd:\n\nCommand Line Tool\n=================\n\nIt is also possible to run a small CLI tool for testing out some\nSQL comm"
},
{
"path": "docs/source/conf.py",
"chars": 2263,
"preview": "# Configuration file for the Sphinx documentation builder.\n#\n# This file only contains a selection of the most common op"
},
{
"path": "docs/source/configuration.rst",
"chars": 844,
"preview": ".. _configuration:\n\nConfiguration in Dask-SQL\n==========================\n\n``dask-sql`` supports a list of configuration "
},
{
"path": "docs/source/custom.rst",
"chars": 3153,
"preview": ".. _custom:\n\nCustom Functions and Aggregations\n=================================\n\nAdditional to the included SQL functio"
},
{
"path": "docs/source/data_input.rst",
"chars": 9856,
"preview": ".. _data_input:\n\nData Loading and Input\n======================\n\nBefore data can be queried with ``dask-sql``, it needs t"
},
{
"path": "docs/source/fugue.rst",
"chars": 2368,
"preview": "FugueSQL Integrations\n=====================\n\n`FugueSQL <https://fugue-tutorials.readthedocs.io/tutorials/fugue_sql/index"
},
{
"path": "docs/source/how_does_it_work.rst",
"chars": 5599,
"preview": "How does it work?\n=================\n\nAt the core, ``dask-sql`` does two things:\n\n- Translates the SQL query using `Apach"
},
{
"path": "docs/source/index.rst",
"chars": 4007,
"preview": "dask-sql\n========\n\n``dask-sql`` is a distributed SQL query engine in Python.\nIt allows you to query and transform your d"
},
{
"path": "docs/source/installation.rst",
"chars": 2974,
"preview": ".. _installation:\n\nInstallation\n============\n\n``dask-sql`` can be installed via ``conda`` (preferred) or ``pip`` - or in"
},
{
"path": "docs/source/machine_learning.rst",
"chars": 11149,
"preview": ".. _machine_learning:\n\nMachine Learning\n================\n\n.. note::\n Machine Learning support is experimental in ``da"
},
{
"path": "docs/source/quickstart.rst",
"chars": 3939,
"preview": ".. _quickstart:\n\nQuickstart\n==========\n\nAfter :ref:`installation`, you can start querying your data using SQL.\n\nRun the "
},
{
"path": "docs/source/server.rst",
"chars": 5134,
"preview": ".. _server:\n\nSQL Server\n==========\n\n``dask-sql`` comes with a small test implementation for a SQL server.\nInstead of reb"
},
{
"path": "docs/source/sql/creation.rst",
"chars": 5682,
"preview": ".. _creation:\n\nTable Creation\n==============\n\nAs described in :ref:`quickstart`, it is possible to register an already\nc"
},
{
"path": "docs/source/sql/describe.rst",
"chars": 4251,
"preview": "Metadata Information\n====================\n\nWith these operations, it is possible to get information on the currently reg"
},
{
"path": "docs/source/sql/ml.rst",
"chars": 7107,
"preview": ".. _ml:\n\nMachine Learning in SQL\n=======================\n\n.. note::\n Machine Learning support is experimental in ``da"
},
{
"path": "docs/source/sql/select.rst",
"chars": 7219,
"preview": ".. _select:\n\nData Retrieval\n==============\n\nQuery data from already created tables. The ``SELECT`` call follows mostly t"
},
{
"path": "docs/source/sql.rst",
"chars": 9027,
"preview": ".. _sql:\n\nSQL Syntax\n==========\n\n``dask-sql`` understands SQL in (mostly) presto SQL syntax.\nSo far, not every valid SQL"
},
{
"path": "notebooks/Custom Functions.ipynb",
"chars": 3800,
"preview": "{\n \"cells\": [\n {\n \"cell_type\": \"markdown\",\n \"metadata\": {},\n \"source\": [\n \"# Custom Functions\"\n ]\n },\n {\n "
},
{
"path": "notebooks/Feature Overview.ipynb",
"chars": 17189,
"preview": "{\n \"cells\": [\n {\n \"attachments\": {},\n \"cell_type\": \"markdown\",\n \"metadata\": {},\n \"source\": [\n \"# Dask-SQL\\n\""
},
{
"path": "notebooks/FugueSQL.ipynb",
"chars": 14539,
"preview": "{\n \"cells\": [\n {\n \"cell_type\": \"markdown\",\n \"id\": \"f39e2dbc-21a1-4d9a-bed7-e2bf2bd25bb8\",\n \"metadata\": {},\n \"so"
},
{
"path": "notebooks/iris.csv",
"chars": 3858,
"preview": "sepal_length,sepal_width,petal_length,petal_width,species\n5.1,3.5,1.4,0.2,setosa\n4.9,3.0,1.4,0.2,setosa\n4.7,3.2,1.3,0.2,"
},
{
"path": "pyproject.toml",
"chars": 3084,
"preview": "[build-system]\nrequires = [\"maturin>=1.3,<1.4\"]\nbuild-backend = \"maturin\"\n\n[project]\nname = \"dask_sql\"\ndescription = \"SQ"
},
{
"path": "rustfmt.toml",
"chars": 103,
"preview": "imports_layout = \"HorizontalVertical\"\nimports_granularity = \"Crate\"\ngroup_imports = \"StdExternalCrate\"\n"
},
{
"path": "setup.cfg",
"chars": 703,
"preview": "[flake8]\n# References:\n# https://flake8.readthedocs.io/en/latest/user/configuration.html\n# https://flake8.readthedocs.io"
},
{
"path": "src/dialect.rs",
"chars": 10523,
"preview": "use core::{iter::Peekable, str::Chars};\n\nuse datafusion_python::datafusion_sql::sqlparser::{\n ast::{Expr, Function, F"
},
{
"path": "src/error.rs",
"chars": 1617,
"preview": "use std::fmt::{Display, Formatter};\n\nuse datafusion_python::{\n datafusion_common::DataFusionError,\n datafusion_sql"
},
{
"path": "src/expression.rs",
"chars": 39061,
"preview": "use std::{borrow::Cow, convert::From, sync::Arc};\n\nuse datafusion_python::{\n datafusion::arrow::datatypes::DataType,\n"
},
{
"path": "src/lib.rs",
"chars": 1373,
"preview": "use log::debug;\nuse pyo3::prelude::*;\n\nmod dialect;\nmod error;\nmod expression;\nmod parser;\nmod sql;\n\n/// Low-level DataF"
},
{
"path": "src/parser.rs",
"chars": 56283,
"preview": "//! SQL Parser\n//!\n//! Declares a SQL parser based on sqlparser that handles custom formats that we need.\n\nuse std::coll"
},
{
"path": "src/sql/column.rs",
"chars": 736,
"preview": "use datafusion_python::datafusion_common::Column;\nuse pyo3::prelude::*;\n\n#[pyclass(name = \"Column\", module = \"dask_sql\","
},
{
"path": "src/sql/exceptions.rs",
"chars": 871,
"preview": "use std::fmt::Debug;\n\nuse pyo3::{create_exception, PyErr};\n\n// Identifies exceptions that occur while attempting to gene"
},
{
"path": "src/sql/function.rs",
"chars": 1086,
"preview": "use std::collections::HashMap;\n\nuse datafusion_python::datafusion::arrow::datatypes::DataType;\nuse pyo3::prelude::*;\n\nus"
},
{
"path": "src/sql/logical/aggregate.rs",
"chars": 4164,
"preview": "use datafusion_python::datafusion_expr::{\n expr::{AggregateFunction, AggregateUDF, Alias},\n logical_plan::{Aggrega"
},
{
"path": "src/sql/logical/alter_schema.rs",
"chars": 3614,
"preview": "use std::{\n any::Any,\n fmt,\n hash::{Hash, Hasher},\n sync::Arc,\n};\n\nuse datafusion_python::{\n datafusion_c"
},
{
"path": "src/sql/logical/alter_table.rs",
"chars": 4055,
"preview": "use std::{\n any::Any,\n fmt,\n hash::{Hash, Hasher},\n sync::Arc,\n};\n\nuse datafusion_python::{\n datafusion_c"
},
{
"path": "src/sql/logical/analyze_table.rs",
"chars": 3811,
"preview": "use std::{\n any::Any,\n fmt,\n hash::{Hash, Hasher},\n sync::Arc,\n};\n\nuse datafusion_python::{\n datafusion_c"
},
{
"path": "src/sql/logical/create_catalog_schema.rs",
"chars": 3859,
"preview": "use std::{\n any::Any,\n fmt,\n hash::{Hash, Hasher},\n sync::Arc,\n};\n\nuse datafusion_python::{\n datafusion_c"
},
{
"path": "src/sql/logical/create_experiment.rs",
"chars": 4875,
"preview": "use std::{\n any::Any,\n fmt,\n hash::{Hash, Hasher},\n sync::Arc,\n};\n\nuse datafusion_python::{\n datafusion_c"
},
{
"path": "src/sql/logical/create_memory_table.rs",
"chars": 3449,
"preview": "use datafusion_python::datafusion_expr::{\n logical_plan::{CreateMemoryTable, CreateView},\n DdlStatement,\n Logic"
},
{
"path": "src/sql/logical/create_model.rs",
"chars": 4653,
"preview": "use std::{\n any::Any,\n fmt,\n hash::{Hash, Hasher},\n sync::Arc,\n};\n\nuse datafusion_python::{\n datafusion_c"
},
{
"path": "src/sql/logical/create_table.rs",
"chars": 4242,
"preview": "use std::{\n any::Any,\n fmt,\n hash::{Hash, Hasher},\n sync::Arc,\n};\n\nuse datafusion_python::{\n datafusion_c"
},
{
"path": "src/sql/logical/describe_model.rs",
"chars": 3469,
"preview": "use std::{\n any::Any,\n fmt,\n hash::{Hash, Hasher},\n sync::Arc,\n};\n\nuse datafusion_python::{\n datafusion_c"
},
{
"path": "src/sql/logical/drop_model.rs",
"chars": 3545,
"preview": "use std::{\n any::Any,\n fmt,\n hash::{Hash, Hasher},\n sync::Arc,\n};\n\nuse datafusion_python::{\n datafusion_c"
},
{
"path": "src/sql/logical/drop_schema.rs",
"chars": 3229,
"preview": "use std::{\n any::Any,\n fmt,\n hash::{Hash, Hasher},\n sync::Arc,\n};\n\nuse datafusion_python::{\n datafusion_c"
},
{
"path": "src/sql/logical/drop_table.rs",
"chars": 956,
"preview": "use datafusion_python::datafusion_expr::{\n logical_plan::{DropTable, LogicalPlan},\n DdlStatement,\n};\nuse pyo3::pre"
},
{
"path": "src/sql/logical/empty_relation.rs",
"chars": 1169,
"preview": "use datafusion_python::datafusion_expr::logical_plan::{EmptyRelation, LogicalPlan};\nuse pyo3::prelude::*;\n\nuse crate::sq"
},
{
"path": "src/sql/logical/explain.rs",
"chars": 978,
"preview": "use datafusion_python::datafusion_expr::{logical_plan::Explain, LogicalPlan};\nuse pyo3::prelude::*;\n\nuse crate::sql::exc"
},
{
"path": "src/sql/logical/export_model.rs",
"chars": 3772,
"preview": "use std::{\n any::Any,\n fmt,\n hash::{Hash, Hasher},\n sync::Arc,\n};\n\nuse datafusion_python::{\n datafusion_c"
},
{
"path": "src/sql/logical/filter.rs",
"chars": 947,
"preview": "use datafusion_python::datafusion_expr::{logical_plan::Filter, LogicalPlan};\nuse pyo3::prelude::*;\n\nuse crate::{expressi"
},
{
"path": "src/sql/logical/join.rs",
"chars": 4400,
"preview": "use datafusion_python::{\n datafusion_common::Column,\n datafusion_expr::{\n and,\n logical_plan::{Join,"
},
{
"path": "src/sql/logical/limit.rs",
"chars": 1282,
"preview": "use datafusion_python::{\n datafusion_common::ScalarValue,\n datafusion_expr::{logical_plan::Limit, Expr, LogicalPla"
},
{
"path": "src/sql/logical/predict_model.rs",
"chars": 3605,
"preview": "use std::{\n any::Any,\n fmt,\n hash::{Hash, Hasher},\n sync::Arc,\n};\n\nuse datafusion_python::{\n datafusion_c"
},
{
"path": "src/sql/logical/projection.rs",
"chars": 2292,
"preview": "use datafusion_python::datafusion_expr::{\n expr::Alias,\n logical_plan::Projection,\n Expr,\n LogicalPlan,\n};\nu"
},
{
"path": "src/sql/logical/repartition_by.rs",
"chars": 1993,
"preview": "use datafusion_python::datafusion_expr::{\n logical_plan::{Partitioning, Repartition},\n Expr,\n LogicalPlan,\n};\nu"
},
{
"path": "src/sql/logical/show_columns.rs",
"chars": 3473,
"preview": "use std::{\n any::Any,\n fmt,\n hash::{Hash, Hasher},\n sync::Arc,\n};\n\nuse datafusion_python::{\n datafusion_c"
},
{
"path": "src/sql/logical/show_models.rs",
"chars": 2942,
"preview": "use std::{\n any::Any,\n fmt,\n hash::{Hash, Hasher},\n sync::Arc,\n};\n\nuse datafusion_python::{\n datafusion_c"
},
{
"path": "src/sql/logical/show_schemas.rs",
"chars": 3382,
"preview": "use std::{\n any::Any,\n fmt,\n hash::{Hash, Hasher},\n sync::Arc,\n};\n\nuse datafusion_python::{\n datafusion_c"
},
{
"path": "src/sql/logical/show_tables.rs",
"chars": 3467,
"preview": "use std::{\n any::Any,\n fmt,\n hash::{Hash, Hasher},\n sync::Arc,\n};\n\nuse datafusion_python::{\n datafusion_c"
},
{
"path": "src/sql/logical/sort.rs",
"chars": 965,
"preview": "use datafusion_python::datafusion_expr::{logical_plan::Sort, LogicalPlan};\nuse pyo3::prelude::*;\n\nuse crate::{\n expre"
},
{
"path": "src/sql/logical/subquery_alias.rs",
"chars": 871,
"preview": "use datafusion_python::datafusion_expr::{logical_plan::SubqueryAlias, LogicalPlan};\nuse pyo3::prelude::*;\n\nuse crate::sq"
},
{
"path": "src/sql/logical/table_scan.rs",
"chars": 10164,
"preview": "use std::{sync::Arc, vec};\n\nuse datafusion_python::{\n datafusion_common::{DFSchema, ScalarValue},\n datafusion_expr"
},
{
"path": "src/sql/logical/use_schema.rs",
"chars": 2983,
"preview": "use std::{\n any::Any,\n fmt,\n hash::{Hash, Hasher},\n sync::Arc,\n};\n\nuse datafusion_python::{\n datafusion_c"
},
{
"path": "src/sql/logical/window.rs",
"chars": 6593,
"preview": "use datafusion_python::{\n datafusion_common::ScalarValue,\n datafusion_expr::{\n expr::WindowFunction,\n "
},
{
"path": "src/sql/logical.rs",
"chars": 17390,
"preview": "use crate::sql::{\n table,\n types::{rel_data_type::RelDataType, rel_data_type_field::RelDataTypeField},\n};\n\npub mod"
},
{
"path": "src/sql/optimizer/decorrelate_where_exists.rs",
"chars": 7374,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one\n// or more contributor license agreements. See the NOTICE"
},
{
"path": "src/sql/optimizer/decorrelate_where_in.rs",
"chars": 9067,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one\n// or more contributor license agreements. See the NOTICE"
},
{
"path": "src/sql/optimizer/dynamic_partition_pruning.rs",
"chars": 44591,
"preview": "//! Optimizer rule for dynamic partition pruning (DPP)\n//!\n//! DPP refers to a query optimization rule in which distinct"
},
{
"path": "src/sql/optimizer/join_reorder.rs",
"chars": 13899,
"preview": "//! Join reordering based on the paper \"Improving Join Reordering for Large Scale Distributed Computing\"\n//! https://iee"
},
{
"path": "src/sql/optimizer/utils.rs",
"chars": 16250,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one\n// or more contributor license agreements. See the NOTICE"
}
]
// ... and 51 more files (download for full content)
About this extraction
This page contains the full source code of the nils-braun/dask-sql GitHub repository, extracted and formatted as plain text for AI agents and large language models (LLMs). The extraction includes 251 files (1.3 MB), approximately 324.2k tokens, and a symbol index with 1675 extracted functions, classes, methods, constants, and types. Use this with OpenClaw, Claude, ChatGPT, Cursor, Windsurf, or any other AI tool that accepts text input. You can copy the full output to your clipboard or download it as a .txt file.
Extracted by GitExtract — free GitHub repo to text converter for AI. Built by Nikandr Surkov.