diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index 736703c551..3926fec306 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -1,7 +1,14 @@ -crates/ @wjones127 @roeap @rtyler @hntd187 @ion-elgreco -delta-inspect/ @wjones127 @rtyler +crates/core @roeap @rtyler @hntd187 @ion-elgreco +crates/deltalake @roeap @rtyler @hntd187 @ion-elgreco +crates/aws @rtyler +crates/lakefs @ion-elgreco +crates/catalog-unity @roeap @hntd187 + +delta-inspect/ @rtyler + proofs/ @houqp -python/ @wjones127 @fvaleye @roeap @ion-elgreco +python/ @wjones127 @roeap @ion-elgreco + tlaplus/ @houqp -.github/ @wjones127 @rtyler -docs/ @MrPowers + +.github/ @rtyler diff --git a/.github/actions/setup-env/action.yml b/.github/actions/setup-env/action.yml index 8339c45449..74f05ea84d 100644 --- a/.github/actions/setup-env/action.yml +++ b/.github/actions/setup-env/action.yml @@ -4,12 +4,12 @@ description: "Set up Python, virtual environment, and Rust toolchain" inputs: python-version: description: "The Python version to set up" - required: true + required: false default: "3.10" rust-toolchain: description: "The Rust toolchain to set up" - required: true + required: false default: "stable" runs: diff --git a/.github/codecov.yml b/.github/codecov.yml index dd93c3b7cf..2bc285fde0 100644 --- a/.github/codecov.yml +++ b/.github/codecov.yml @@ -1,17 +1,18 @@ - coverage: status: project: default: # allow some leniency on the deviation of pull requests - threshold: '1%' - informational: true + threshold: 1 + if_ci_failed: error + informational: false patch: default: - informational: true - + if_ci_failed: error + informational: false ignore: - "delta-inspect/" - "proofs/" - "**/*.toml" + - "crates/benchmarks/" diff --git a/.github/scripts/retry_integration_test.sh b/.github/scripts/retry_integration_test.sh index cac54489da..0708f89a01 100755 --- a/.github/scripts/retry_integration_test.sh +++ b/.github/scripts/retry_integration_test.sh @@ -5,7 +5,7 @@ MAX_RETRIES=$2 RETRY_DELAY=$3 ATTEMPT=1 run_command() { - uv run --no-sync pytest -m "($TEST_NAME and integration)" --doctest-modules 2>&1 + uv run --no-sync pytest -m "($TEST_NAME and integration and pyarrow)" --doctest-modules 2>&1 } until [ $ATTEMPT -gt $MAX_RETRIES ] do diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index ba23352e66..229e7ba3c4 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -14,89 +14,80 @@ env: # Disable incremental builds by cargo for CI which should save disk space # and hopefully avoid final link "No space left on device" CARGO_INCREMENTAL: 0 + SCCACHE_GHA_ENABLED: "true" + RUSTC_WRAPPER: "sccache" jobs: - default_build: - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v3 - - - name: Install minimal stable with clippy and rustfmt - uses: actions-rs/toolchain@v1 - with: - profile: default - toolchain: '1.82' - override: true - - - name: Build - run: (cd crates/deltalake && cargo build) - - format: - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v3 - - - name: Install minimal stable with clippy and rustfmt - uses: actions-rs/toolchain@v1 - with: - profile: default - toolchain: '1.82' - override: true - - - name: Format - run: cargo fmt -- --check - + # run various build comnfigurations, fmt, and clippy. build: strategy: - fail-fast: false + fail-fast: true matrix: os: - ubuntu-latest - windows-latest + - macos-latest runs-on: ${{ matrix.os }} steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 + + - name: Run sccache-cache + uses: mozilla-actions/sccache-action@v0.0.9 - name: Install minimal stable with clippy and rustfmt uses: actions-rs/toolchain@v1 with: profile: default - toolchain: '1.82' + toolchain: "1.82" override: true + - name: Format + run: cargo fmt -- --check + + - name: Default build + run: (cd crates/deltalake && cargo build --tests) + - name: build and lint with clippy run: cargo clippy --features ${{ env.DEFAULT_FEATURES }} --tests - name: Spot-check build for native-tls features run: cargo clippy --no-default-features --features azure,datafusion,s3-native-tls,gcs,glue --tests - - name: Check docs - run: cargo doc --features ${{ env.DEFAULT_FEATURES }} - - name: Check no default features (except rustls) run: cargo check --no-default-features --features rustls - test: + - name: Check docs + run: cargo doc --no-deps --features ${{ env.DEFAULT_FEATURES }} + + unit_test: + name: Unit Tests strategy: - fail-fast: false + fail-fast: true matrix: os: - ubuntu-latest - windows-latest + - macos-latest runs-on: ${{ matrix.os }} + steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 + + - name: Run sccache-cache + uses: mozilla-actions/sccache-action@v0.0.9 - name: Install minimal stable with clippy and rustfmt uses: actions-rs/toolchain@v1 with: profile: default - toolchain: '1.82' + toolchain: "1.82" override: true - name: Run tests - run: cargo test --verbose --features ${{ env.DEFAULT_FEATURES }} + run: | + make setup-dat + cargo test --features ${{ env.DEFAULT_FEATURES }} integration_test: name: Integration Tests @@ -115,13 +106,71 @@ jobs: AZURE_STORAGE_CONNECTION_STRING: "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://localhost:10000/devstoreaccount1;QueueEndpoint=http://localhost:10001/devstoreaccount1;" steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 + + - name: Run sccache-cache + uses: mozilla-actions/sccache-action@v0.0.9 + + - name: Install minimal stable with clippy and rustfmt + uses: actions-rs/toolchain@v1 + with: + profile: default + toolchain: "1.82" + override: true + + - name: Install cargo-llvm-cov + uses: taiki-e/install-action@cargo-llvm-cov + + - name: Start emulated services + run: docker compose up -d + + - name: Run tests with rustls (default) + run: | + gmake setup-dat + cargo llvm-cov \ + --features integration_test,${{ env.DEFAULT_FEATURES }} \ + --workspace \ + --exclude delta-inspect \ + --exclude deltalake-hdfs \ + --exclude deltalake-lakefs \ + --codecov \ + --output-path codecov.json + + - name: Upload coverage to Codecov + uses: codecov/codecov-action@v4 + with: + files: codecov.json + fail_ci_if_error: true + env: + CODECOV_TOKEN: ${{ secrets.CODECOV_TOKEN }} + + integration_test_native_tls: + name: Integration Tests (Native TLS) + runs-on: ubuntu-latest + env: + # https://github.com/rust-lang/cargo/issues/10280 + CARGO_NET_GIT_FETCH_WITH_CLI: "true" + AWS_DEFAULT_REGION: "us-east-1" + AWS_ACCESS_KEY_ID: deltalake + AWS_SECRET_ACCESS_KEY: weloverust + AWS_ENDPOINT_URL: http://localhost:4566 + AWS_ALLOW_HTTP: "1" + AZURE_USE_EMULATOR: "1" + AZURE_STORAGE_ALLOW_HTTP: "1" + AZURITE_BLOB_STORAGE_URL: "http://localhost:10000" + AZURE_STORAGE_CONNECTION_STRING: "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://localhost:10000/devstoreaccount1;QueueEndpoint=http://localhost:10001/devstoreaccount1;" + + steps: + - uses: actions/checkout@v4 + + - name: Run sccache-cache + uses: mozilla-actions/sccache-action@v0.0.9 - name: Install minimal stable with clippy and rustfmt uses: actions-rs/toolchain@v1 with: profile: default - toolchain: '1.82' + toolchain: "1.82" override: true # Install Java and Hadoop for HDFS integration tests @@ -139,15 +188,63 @@ jobs: - name: Start emulated services run: docker compose up -d - - name: Run tests with rustls (default) - run: | - cargo test --features integration_test,${{ env.DEFAULT_FEATURES }} - - name: Run tests with native-tls run: | - cargo clean + gmake setup-dat cargo test --no-default-features --features integration_test,s3-native-tls,datafusion + integration_test_hdfs: + name: Integration Tests (HDFS) + runs-on: ubuntu-latest + env: + # https://github.com/rust-lang/cargo/issues/10280 + CARGO_NET_GIT_FETCH_WITH_CLI: "true" + + steps: + - uses: actions/checkout@v4 + + - name: Run sccache-cache + uses: mozilla-actions/sccache-action@v0.0.9 + + - name: Install minimal stable with clippy and rustfmt + uses: actions-rs/toolchain@v1 + with: + profile: default + toolchain: "1.82" + override: true + + - name: Install cargo-llvm-cov + uses: taiki-e/install-action@cargo-llvm-cov + + # Install Java and Hadoop for HDFS integration tests + - uses: actions/setup-java@v4 + with: + distribution: "temurin" + java-version: "17" + + - name: Download Hadoop + run: | + wget -q https://dlcdn.apache.org/hadoop/common/hadoop-3.4.0/hadoop-3.4.0.tar.gz + tar -xf hadoop-3.4.0.tar.gz -C $GITHUB_WORKSPACE + echo "$GITHUB_WORKSPACE/hadoop-3.4.0/bin" >> $GITHUB_PATH + + - name: Run tests with rustls (default) + run: | + gmake setup-dat + cargo llvm-cov \ + --features integration_test \ + --package deltalake-hdfs \ + --codecov \ + --output-path codecov.json + + - name: Upload coverage to Codecov + uses: codecov/codecov-action@v4 + with: + files: codecov.json + fail_ci_if_error: true + env: + CODECOV_TOKEN: ${{ secrets.CODECOV_TOKEN }} + integration_test_lakefs: name: Integration Tests (LakeFS v1.48) runs-on: ubuntu-latest @@ -156,15 +253,21 @@ jobs: CARGO_NET_GIT_FETCH_WITH_CLI: "true" steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 + + - name: Run sccache-cache + uses: mozilla-actions/sccache-action@v0.0.9 - name: Install minimal stable with clippy and rustfmt uses: actions-rs/toolchain@v1 with: profile: default - toolchain: '1.82' + toolchain: "1.82" override: true + - name: Install cargo-llvm-cov + uses: taiki-e/install-action@cargo-llvm-cov + - name: Download Lakectl run: | wget -q https://github.com/treeverse/lakeFS/releases/download/v1.48.1/lakeFS_1.48.1_Linux_x86_64.tar.gz @@ -176,5 +279,17 @@ jobs: - name: Run tests with rustls (default) run: | - cargo test --features integration_test_lakefs,lakefs,datafusion - + gmake setup-dat + cargo llvm-cov \ + --package deltalake-lakefs \ + --features integration_test_lakefs \ + --codecov \ + --output-path codecov.json + + - name: Upload coverage to Codecov + uses: codecov/codecov-action@v4 + with: + files: codecov.json + fail_ci_if_error: true + env: + CODECOV_TOKEN: ${{ secrets.CODECOV_TOKEN }} diff --git a/.github/workflows/codecov.yml b/.github/workflows/codecov.yml deleted file mode 100644 index a790cea087..0000000000 --- a/.github/workflows/codecov.yml +++ /dev/null @@ -1,36 +0,0 @@ -name: coverage - -on: - push: - branches: [main, "rust-v*"] - pull_request: - branches: [main, "rust-v*"] - -env: - DEFAULT_FEATURES: "azure,datafusion,s3,gcs,glue,hdfs " - -jobs: - coverage: - runs-on: ubuntu-latest - env: - CARGO_TERM_COLOR: always - steps: - - uses: actions/checkout@v4 - - name: Install rust - uses: actions-rs/toolchain@v1 - with: - profile: default - toolchain: '1.82' - override: true - - name: Install cargo-llvm-cov - uses: taiki-e/install-action@cargo-llvm-cov - - uses: Swatinem/rust-cache@v2 - - name: Generate code coverage - run: cargo llvm-cov --features ${DEFAULT_FEATURES} --workspace --codecov --output-path codecov.json -- --skip read_table_version_hdfs --skip test_read_tables_lakefs - - name: Upload coverage to Codecov - uses: codecov/codecov-action@v4 - with: - files: codecov.json - fail_ci_if_error: true - env: - CODECOV_TOKEN: ${{ secrets.CODECOV_TOKEN }} diff --git a/.github/workflows/dev_pr.yml b/.github/workflows/dev_pr.yml index 3b4588dfb0..afea36b0ce 100644 --- a/.github/workflows/dev_pr.yml +++ b/.github/workflows/dev_pr.yml @@ -17,7 +17,7 @@ jobs: contents: read pull-requests: write steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 - name: Assign GitHub labels if: | @@ -54,7 +54,7 @@ jobs: with: script: | const message = `**ACTION NEEDED** - + delta-rs follows the [Conventional Commits\ specification](https://www.conventionalcommits.org/en/v1.0.0/) for\ release automation. @@ -82,3 +82,11 @@ jobs: body: message }) core.setFailed(message) + + typos: + name: Spell Check + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v4 + - name: Check spelling + uses: crate-ci/typos@v1 diff --git a/.github/workflows/docs.yml b/.github/workflows/docs.yml index 5729b87624..7c7ea7b46c 100644 --- a/.github/workflows/docs.yml +++ b/.github/workflows/docs.yml @@ -14,7 +14,7 @@ on: # Allow manual trigger for now workflow_dispatch: -env: +env: IS_RELEASE: ${{ github.event_name == 'repository_dispatch' || github.event_name == 'workflow_dispatch' }} BUILD_ARGS: ${{ (github.event_name == 'repository_dispatch' || github.event_name == 'workflow_dispatch') && 'install MATURIN_EXTRA_ARGS="--manylinux off"' || 'develop' }} @@ -22,31 +22,27 @@ jobs: markdown-link-check: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v3 - - uses: gaurav-nelson/github-action-markdown-link-check@v1 - with: - config-file: docs/mlc-config.json - folder-path: docs + - uses: actions/checkout@v4 + - uses: gaurav-nelson/github-action-markdown-link-check@v1 + with: + config-file: docs/mlc-config.json + folder-path: docs lint: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 - run: | cd docs make check build-deploy: - needs: - [ - lint, - markdown-link-check, - ] + needs: [lint, markdown-link-check] runs-on: ubuntu-latest permissions: contents: write steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 - name: Setup Environment uses: ./.github/actions/setup-env @@ -80,5 +76,3 @@ jobs: git-config-name: "Github Action" git-config-email: "deltars@users.noreply.github.com" commit-message: "doc update for tag `${{ github.event.client_payload.tag || github.ref_name }}`" - - diff --git a/.github/workflows/python_benchmark.yml b/.github/workflows/python_benchmark.yml index c065bfd91e..74f2d6248b 100644 --- a/.github/workflows/python_benchmark.yml +++ b/.github/workflows/python_benchmark.yml @@ -1,12 +1,9 @@ name: python_benchmark - # This is separate from the python_build so that it doesn't need to run on the merge group on: push: branches: [main] - pull_request: - branches: [main] defaults: run: @@ -21,7 +18,7 @@ jobs: CARGO_INCREMENTAL: 0 steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Setup Environment uses: ./.github/actions/setup-env @@ -39,7 +36,7 @@ jobs: - name: Run benchmark run: | - uv run pytest tests/test_benchmark.py -m benchmark --benchmark-json output.json + uv run pytest tests/test_benchmark.py -m '(benchmark and pyarrow)' --benchmark-json output.json - name: Store benchmark result uses: benchmark-action/github-action-benchmark@v1 @@ -48,4 +45,3 @@ jobs: output-file-path: python/output.json external-data-json-path: ./cache/benchmark-data.json fail-on-alert: true - diff --git a/.github/workflows/python_build.yml b/.github/workflows/python_build.yml index 6cf949c309..5aa6098313 100644 --- a/.github/workflows/python_build.yml +++ b/.github/workflows/python_build.yml @@ -11,45 +11,39 @@ defaults: run: working-directory: ./python +env: + RUSTFLAGS: "-C debuginfo=line-tables-only" + CARGO_INCREMENTAL: 0 + jobs: - lint: + test-minimal: runs-on: ubuntu-latest + env: + SCCACHE_GHA_ENABLED: "true" + RUSTC_WRAPPER: "sccache" steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 + - name: Run sccache-cache + uses: mozilla-actions/sccache-action@v0.0.9 - name: Setup Environment uses: ./.github/actions/setup-env + with: + python-version: 3.9 - name: Check Python run: | - uv sync --only-group dev --no-install-project + uv sync --no-install-project make check-python - name: Check Rust run: make check-rust - test-minimal: - name: Python Build (Python 3.9 PyArrow 16.0.0) - runs-on: ubuntu-latest - env: - RUSTFLAGS: "-C debuginfo=line-tables-only" - CARGO_INCREMENTAL: 0 - - steps: - - uses: actions/checkout@v3 - - - name: Setup Environment - uses: ./.github/actions/setup-env - with: - python-version: 3.9 - - name: Build and install deltalake run: | # Install minimum PyArrow version make develop uv pip install pyarrow==16.0.0 - env: - RUSTFLAGS: "-C debuginfo=line-tables-only" - name: Run tests run: make unit-test @@ -58,11 +52,13 @@ jobs: name: Python Build (Python 3.10 PyArrow latest) runs-on: ubuntu-latest env: - RUSTFLAGS: "-C debuginfo=1" - CARGO_INCREMENTAL: 0 + SCCACHE_GHA_ENABLED: "true" + RUSTC_WRAPPER: "sccache" steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 + - name: Run sccache-cache + uses: mozilla-actions/sccache-action@v0.0.9 - name: Setup Environment uses: ./.github/actions/setup-env @@ -73,27 +69,25 @@ jobs: - name: Build and install deltalake run: make develop - - name: Download Data Acceptance Tests (DAT) files - run: make setup-dat - - name: Run tests - run: uv run --no-sync pytest -m '((s3 or azure) and integration) or not integration and not benchmark' --doctest-modules + run: uv run --no-sync pytest -m '((s3 or azure) and integration) or not integration and not benchmark and not no_pyarrow' --doctest-modules - name: Test without pandas run: | uv pip uninstall pandas - uv run --no-sync pytest -m "not pandas and not integration and not benchmark" - uv pip install pandas + uv run --no-sync pytest -m "not pandas and not integration and not benchmark and not no_pyarrow" + + - name: Test without pyarrow and without pandas + run: | + uv pip uninstall pyarrow + uv run --no-sync pytest -m "not pyarrow and not pandas and not integration and not benchmark" test-lakefs: name: Python Build (Python 3.10 LakeFS Integration tests) runs-on: ubuntu-latest - env: - RUSTFLAGS: "-C debuginfo=1" - CARGO_INCREMENTAL: 0 steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 - name: Setup Environment uses: ./.github/actions/setup-env @@ -104,21 +98,15 @@ jobs: - name: Build and install deltalake run: make develop - - name: Download Data Acceptance Tests (DAT) files - run: make setup-dat - - name: Run tests run: uv run --no-sync pytest -m '(lakefs and integration)' --doctest-modules test-unitycatalog-databricks: name: Python Build (Python 3.10 Unity Catalog Integration tests) runs-on: ubuntu-latest - env: - RUSTFLAGS: "-C debuginfo=1" - CARGO_INCREMENTAL: 0 steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 - name: Setup Environment uses: ./.github/actions/setup-env @@ -133,21 +121,15 @@ jobs: - name: Build and install deltalake run: make develop - - name: Download Data Acceptance Tests (DAT) files - run: make setup-dat - - name: Run tests run: ../../delta-rs/.github/scripts/retry_integration_test.sh unitycatalog_databricks 5 10 test-unitycatalog-oss: name: Python Build (Python 3.10 Unity Catalog Integration tests) runs-on: ubuntu-latest - env: - RUSTFLAGS: "-C debuginfo=1" - CARGO_INCREMENTAL: 0 steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 - name: Setup Environment uses: ./.github/actions/setup-env @@ -162,21 +144,15 @@ jobs: - name: Build and install deltalake run: make develop - - name: Download Data Acceptance Tests (DAT) files - run: make setup-dat - - name: Run tests run: ../../delta-rs/.github/scripts/retry_integration_test.sh unitycatalog_oss 5 10 test-pyspark: name: PySpark Integration Tests runs-on: ubuntu-latest - env: - RUSTFLAGS: "-C debuginfo=line-tables-only" - CARGO_INCREMENTAL: 0 steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 - name: Setup Environment uses: ./.github/actions/setup-env @@ -204,7 +180,7 @@ jobs: python-version: ["3.9", "3.10", "3.11", "3.12"] steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 - name: Setup Environment uses: ./.github/actions/setup-env @@ -217,4 +193,19 @@ jobs: - name: Run deltalake run: | - uv run python -c 'import deltalake' + uv run --no-sync python -c 'import deltalake' + + - name: Run deltalake without pyarrow + run: | + uv pip uninstall pyarrow + uv run --no-sync python -c 'import deltalake' + + - name: Run deltalake without pyarrow pandas + run: | + uv pip uninstall pyarrow pandas + uv run --no-sync python -c 'import deltalake' + + - name: Run deltalake without pandas + run: | + uv pip install pyarrow + uv run --no-sync python -c 'import deltalake' diff --git a/.github/workflows/python_release.yml b/.github/workflows/python_release.yml index 6574451133..10ca82d2d9 100644 --- a/.github/workflows/python_release.yml +++ b/.github/workflows/python_release.yml @@ -11,9 +11,9 @@ defaults: jobs: validate-release-tag: name: Validate git tag - runs-on: ubuntu-20.04 + runs-on: ubuntu-latest steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 - name: compare git tag with cargo metadata run: | PUSHED_TAG=${GITHUB_REF##*/} @@ -32,7 +32,7 @@ jobs: target: [x86_64-apple-darwin, aarch64-apple-darwin] runs-on: macos-14 steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 - name: Publish to pypi (without sdist) uses: messense/maturin-action@v1 @@ -48,7 +48,7 @@ jobs: name: PyPI release on Windows runs-on: windows-latest steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 - name: Publish to pypi (without sdist) uses: messense/maturin-action@v1 @@ -64,7 +64,7 @@ jobs: name: PyPI release manylinux-2_17 x86_64 runs-on: ubuntu-latest steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 - name: Publish manylinux to pypi x86_64 (with sdist) uses: messense/maturin-action@v1 @@ -80,10 +80,10 @@ jobs: release-pypi-manylinux-217-aarch64: needs: validate-release-tag - name: PyPI release manylinux-2_17 aarch64 + name: PyPI release manylinux-2_17 aarch64 runs-on: ubuntu-latest steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 - name: Publish manylinux to pypi aarch64 (without sdist) uses: messense/maturin-action@v1 @@ -103,7 +103,7 @@ jobs: name: PyPI release manylinux-2_28 aarch64 runs-on: ubuntu-latest steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 - name: Publish manylinux to pypi aarch64 manylinux-2_28 (without sdist) uses: messense/maturin-action@v1 @@ -136,4 +136,4 @@ jobs: client-payload: > { "tag": "${{ github.ref_name }}" - } \ No newline at end of file + } diff --git a/.github/workflows/rust_release.yml b/.github/workflows/rust_release.yml index a174130b9f..35d0b0cac5 100644 --- a/.github/workflows/rust_release.yml +++ b/.github/workflows/rust_release.yml @@ -11,9 +11,10 @@ defaults: jobs: validate-release-tag: name: Validate git tag - runs-on: ubuntu-20.04 + runs-on: ubuntu-latest steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 + - name: compare git tag with cargo metadata run: | PUSHED_TAG=${GITHUB_REF##*/} @@ -26,11 +27,10 @@ jobs: release-crate: needs: validate-release-tag name: Release crate - strategy: - fail-fast: false - runs-on: ubuntu-20.04 + runs-on: ubuntu-latest + steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 - uses: actions-rs/toolchain@v1 with: diff --git a/.gitignore b/.gitignore index 18dcc39f69..06425842e4 100644 --- a/.gitignore +++ b/.gitignore @@ -23,13 +23,17 @@ __blobstorage__ .githubchangeloggenerator.cache/ .githubchangeloggenerator* data +.zed/ # Add all Cargo.lock files except for those in binary crates Cargo.lock !/aws/delta-checkpoint/Cargo.lock !/delta-inspect/Cargo.lock -!/proofs/Cargo.lock +!/proofs/stateright/Cargo.lock justfile site -__pycache__ \ No newline at end of file +__pycache__ +.zed +.zed/ +dat/ diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml new file mode 100644 index 0000000000..1303eaba8c --- /dev/null +++ b/.pre-commit-config.yaml @@ -0,0 +1,5 @@ +repos: + - repo: https://github.com/crate-ci/typos + rev: v1.32.0 + hooks: + - id: typos diff --git a/CHANGELOG.md b/CHANGELOG.md index 8226868f8b..a9720ad9a7 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -78,7 +78,7 @@ - chore: remove some `file_actions` call sites [\#2787](https://github.com/delta-io/delta-rs/pull/2787) ([roeap](https://github.com/roeap)) - style: more consistent imports [\#2786](https://github.com/delta-io/delta-rs/pull/2786) ([roeap](https://github.com/roeap)) - feat\(python, rust\): added statistics\_truncate\_length in WriterProperties [\#2784](https://github.com/delta-io/delta-rs/pull/2784) ([sherlockbeard](https://github.com/sherlockbeard)) -- fix: pin maturin verison [\#2778](https://github.com/delta-io/delta-rs/pull/2778) ([ion-elgreco](https://github.com/ion-elgreco)) +- fix: pin maturin version [\#2778](https://github.com/delta-io/delta-rs/pull/2778) ([ion-elgreco](https://github.com/ion-elgreco)) - fix: trim trailing slash in url storage options \(\#2656\) [\#2775](https://github.com/delta-io/delta-rs/pull/2775) ([omkar-foss](https://github.com/omkar-foss)) - chore: update the changelog with the 0.19.0 release [\#2774](https://github.com/delta-io/delta-rs/pull/2774) ([rtyler](https://github.com/rtyler)) - feat\(python, rust\): `add feature` operation [\#2712](https://github.com/delta-io/delta-rs/pull/2712) ([ion-elgreco](https://github.com/ion-elgreco)) @@ -184,7 +184,7 @@ - Failure to delete dir and files [\#2703](https://github.com/delta-io/delta-rs/issues/2703) - DeltaTable.from\_data\_catalog not working [\#2699](https://github.com/delta-io/delta-rs/issues/2699) - Project should use the same version of `ruff` in the `lint` stage of `python_build.yml` as in `pyproject.toml` [\#2678](https://github.com/delta-io/delta-rs/issues/2678) -- un-tracked columns are giving json error when pyarrow schema have feild with nullable=False and create\_checkpoint is trigged [\#2675](https://github.com/delta-io/delta-rs/issues/2675) +- un-tracked columns are giving json error when pyarrow schema have field with nullable=False and create\_checkpoint is triggered [\#2675](https://github.com/delta-io/delta-rs/issues/2675) - \[BUG\]write\_delta\({'custom\_metadata':str}\) cannot be converted. str to pyDict error \(0.18.2\_DeltaPython/Windows10\) [\#2697](https://github.com/delta-io/delta-rs/issues/2697) - Pyarrow engine not supporting schema overwrite with Append mode [\#2654](https://github.com/delta-io/delta-rs/issues/2654) - `deltalake-core` version re-exported by `deltalake` different than versions used by `deltalake-azure` and `deltalake-gcp` [\#2647](https://github.com/delta-io/delta-rs/issues/2647) @@ -323,7 +323,7 @@ - Failure to delete dir and files [\#2703](https://github.com/delta-io/delta-rs/issues/2703) - DeltaTable.from\_data\_catalog not working [\#2699](https://github.com/delta-io/delta-rs/issues/2699) - Project should use the same version of `ruff` in the `lint` stage of `python_build.yml` as in `pyproject.toml` [\#2678](https://github.com/delta-io/delta-rs/issues/2678) -- un-tracked columns are giving json error when pyarrow schema have feild with nullable=False and create\_checkpoint is trigged [\#2675](https://github.com/delta-io/delta-rs/issues/2675) +- un-tracked columns are giving json error when pyarrow schema have field with nullable=False and create\_checkpoint is triggered [\#2675](https://github.com/delta-io/delta-rs/issues/2675) - \[BUG\]write\_delta\({'custom\_metadata':str}\) cannot be converted. str to pyDict error \(0.18.2\_DeltaPython/Windows10\) [\#2697](https://github.com/delta-io/delta-rs/issues/2697) - Pyarrow engine not supporting schema overwrite with Append mode [\#2654](https://github.com/delta-io/delta-rs/issues/2654) - `deltalake-core` version re-exported by `deltalake` different than versions used by `deltalake-azure` and `deltalake-gcp` [\#2647](https://github.com/delta-io/delta-rs/issues/2647) @@ -336,7 +336,7 @@ - Unable to read delta table created using Uniform [\#2578](https://github.com/delta-io/delta-rs/issues/2578) - schema merging doesn't work when overwriting with a predicate [\#2567](https://github.com/delta-io/delta-rs/issues/2567) - Not working in AWS Lambda \(0.16.2 - 0.17.4\) OSError: Generic S3 error [\#2511](https://github.com/delta-io/delta-rs/issues/2511) -- DataFusion filter on partition column doesn't work. \(when the phsical schema ordering is different to logical one\) [\#2494](https://github.com/delta-io/delta-rs/issues/2494) +- DataFusion filter on partition column doesn't work. \(when the physical schema ordering is different to logical one\) [\#2494](https://github.com/delta-io/delta-rs/issues/2494) - Creating checkpoints for tables with missing column stats results in Err [\#2493](https://github.com/delta-io/delta-rs/issues/2493) - Cannot merge to a table with a timestamp column after upgrading delta-rs [\#2478](https://github.com/delta-io/delta-rs/issues/2478) - Azure AD Auth fails on ARM64 [\#2475](https://github.com/delta-io/delta-rs/issues/2475) @@ -560,7 +560,7 @@ - Limit concurrent ObjectStore access to avoid resource limitations in constrained environments [\#2457](https://github.com/delta-io/delta-rs/issues/2457) - How to get a DataFrame in Rust? [\#2404](https://github.com/delta-io/delta-rs/issues/2404) -- Allow checkpoint creation when partion column is "timestampNtz " [\#2381](https://github.com/delta-io/delta-rs/issues/2381) +- Allow checkpoint creation when partition column is "timestampNtz " [\#2381](https://github.com/delta-io/delta-rs/issues/2381) - is there a way to make writing timestamp\_ntz optional [\#2339](https://github.com/delta-io/delta-rs/issues/2339) - Update arrow dependency [\#2328](https://github.com/delta-io/delta-rs/issues/2328) - Release GIL in deltalake.write\_deltalake [\#2234](https://github.com/delta-io/delta-rs/issues/2234) @@ -574,7 +574,7 @@ - Issue writing to mounted storage in AKS using delta-rs library [\#2445](https://github.com/delta-io/delta-rs/issues/2445) - TableMerger - when\_matched\_delete\(\) fails when Column names contain special characters [\#2438](https://github.com/delta-io/delta-rs/issues/2438) - Generic DeltaTable error: External error: Arrow error: Invalid argument error: arguments need to have the same data type - while merge data in to delta table [\#2423](https://github.com/delta-io/delta-rs/issues/2423) -- Merge on predicate throw error on date colum: Unable to convert expression to string [\#2420](https://github.com/delta-io/delta-rs/issues/2420) +- Merge on predicate throw error on date column: Unable to convert expression to string [\#2420](https://github.com/delta-io/delta-rs/issues/2420) - Writing Tables with Append mode errors if the schema metadata is different [\#2419](https://github.com/delta-io/delta-rs/issues/2419) - Logstore issues on AWS Lambda [\#2410](https://github.com/delta-io/delta-rs/issues/2410) - Datafusion timestamp type doesn't respect delta lake schema [\#2408](https://github.com/delta-io/delta-rs/issues/2408) @@ -636,12 +636,12 @@ - Error when parsing delete expressions [\#2187](https://github.com/delta-io/delta-rs/issues/2187) - terminate called without an active exception [\#2184](https://github.com/delta-io/delta-rs/issues/2184) - Now conda-installable on M1 [\#2178](https://github.com/delta-io/delta-rs/issues/2178) -- Add error message for parition\_by check [\#2177](https://github.com/delta-io/delta-rs/issues/2177) +- Add error message for partition\_by check [\#2177](https://github.com/delta-io/delta-rs/issues/2177) - deltalake 0.15.2 prints partitions\_values and paths which is not desired [\#2176](https://github.com/delta-io/delta-rs/issues/2176) - cleanup\_metadata can potentially delete most recent checkpoint, corrupting table [\#2174](https://github.com/delta-io/delta-rs/issues/2174) - Broken filter for newly created delta table [\#2169](https://github.com/delta-io/delta-rs/issues/2169) - Hash for StructField should consider more than the name [\#2045](https://github.com/delta-io/delta-rs/issues/2045) -- Schema comparaison in writer [\#1853](https://github.com/delta-io/delta-rs/issues/1853) +- Schema comparison in writer [\#1853](https://github.com/delta-io/delta-rs/issues/1853) - fix\(python\): sort before schema comparison [\#2209](https://github.com/delta-io/delta-rs/pull/2209) ([ion-elgreco](https://github.com/ion-elgreco)) - fix: prevent writing checkpoints with a version that does not exist in table state [\#1863](https://github.com/delta-io/delta-rs/pull/1863) ([rtyler](https://github.com/rtyler)) @@ -898,7 +898,7 @@ Because locking is required to ensure safe cconsistent writes, **there is no ite - Unable to read s3 table: `Unknown scheme: s3` [\#2065](https://github.com/delta-io/delta-rs/issues/2065) - write\_deltalake not respecting writer\_properties [\#2064](https://github.com/delta-io/delta-rs/issues/2064) - Unable to read/write tables with the "gs" schema in the table\_uri in 0.15.1 [\#2060](https://github.com/delta-io/delta-rs/issues/2060) -- LockClient requiered error for S3 backend in 0.15.1 python [\#2057](https://github.com/delta-io/delta-rs/issues/2057) +- LockClient required error for S3 backend in 0.15.1 python [\#2057](https://github.com/delta-io/delta-rs/issues/2057) - Error while writing Pandas DataFrame to Delta Lake \(S3\) [\#2051](https://github.com/delta-io/delta-rs/issues/2051) - Error with dynamo locking provider on 0.15 [\#2034](https://github.com/delta-io/delta-rs/issues/2034) - Conda version 0.15.0 is missing files [\#2021](https://github.com/delta-io/delta-rs/issues/2021) @@ -1007,7 +1007,7 @@ Because locking is required to ensure safe cconsistent writes, **there is no ite - delta-rs does not appear to support tables with liquid clustering [\#1626](https://github.com/delta-io/delta-rs/issues/1626) - Internal Parquet panic when using a Map type. [\#1619](https://github.com/delta-io/delta-rs/issues/1619) - partition\_by with "$" on local filesystem [\#1591](https://github.com/delta-io/delta-rs/issues/1591) -- ProtocolChanged error when perfoming append write [\#1585](https://github.com/delta-io/delta-rs/issues/1585) +- ProtocolChanged error when performing append write [\#1585](https://github.com/delta-io/delta-rs/issues/1585) - Unable to `cargo update` using git tag or rev on Rust 1.70 [\#1580](https://github.com/delta-io/delta-rs/issues/1580) - NoMetadata error when reading detlatable [\#1562](https://github.com/delta-io/delta-rs/issues/1562) - Cannot read delta table: `Delta protocol violation` [\#1557](https://github.com/delta-io/delta-rs/issues/1557) @@ -1293,7 +1293,7 @@ Because locking is required to ensure safe cconsistent writes, **there is no ite - hdfs support [\#300](https://github.com/delta-io/delta-rs/issues/300) - Add decimal primitive type to document [\#1280](https://github.com/delta-io/delta-rs/issues/1280) -- Improve error message when filtering on non-existant partition columns [\#1218](https://github.com/delta-io/delta-rs/issues/1218) +- Improve error message when filtering on non-existent partition columns [\#1218](https://github.com/delta-io/delta-rs/issues/1218) **Fixed bugs:** @@ -1327,7 +1327,7 @@ Because locking is required to ensure safe cconsistent writes, **there is no ite - File pruning does not occur on partition columns [\#1175](https://github.com/delta-io/delta-rs/issues/1175) - Bug: Error loading Delta table locally [\#1157](https://github.com/delta-io/delta-rs/issues/1157) -- Deltalake 0.7.0 with s3 feature compliation error due to rusoto_dynamodb version conflict [\#1191](https://github.com/delta-io/delta-rs/issues/1191) +- Deltalake 0.7.0 with s3 feature compilation error due to rusoto_dynamodb version conflict [\#1191](https://github.com/delta-io/delta-rs/issues/1191) - Writing from a Delta table scan using WriteBuilder fails due to missing object store [\#1186](https://github.com/delta-io/delta-rs/issues/1186) **Merged pull requests:** diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index aa75692808..162fe9faba 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -60,7 +60,7 @@ make build-docs *These are just some basic steps/components to get you started, there are many other very useful extensions for VSCode* -- For a better Rust development experience, install [rust extention](https://marketplace.visualstudio.com/items?itemName=1YiB.rust-bundle) +- For a better Rust development experience, install [rust extension](https://marketplace.visualstudio.com/items?itemName=1YiB.rust-bundle) - For debugging Rust code, install [CodeLLDB](https://marketplace.visualstudio.com/items?itemName=vadimcn.vscode-lldb). The extension should even create Debug launch configurations for the project if you allow it, an easy way to get started. Just set a breakpoint and run the relevant configuration. - For debugging from Python into Rust, follow this procedure: 1. Add this to `.vscode/launch.json` @@ -82,9 +82,9 @@ make build-docs 2. Add a `breakpoint()` statement somewhere in your Python code (main function or at any point in Python code you know will be executed when you run it) 3. Add a breakpoint in Rust code in VSCode editor where you want to drop into the debugger 4. Run the relevant Python code function in your terminal, execution should drop into the Python debugger showing `PDB` prompt -5. Run the following in that promt to get the Python process ID: `import os; os.getpid()` +5. Run the following in that prompt to get the Python process ID: `import os; os.getpid()` 6. Run the `LLDB Attach to Python` from the `Run and Debug` panel of VSCode. This will prompt you for a Process ID to attach to, enter the Python process ID obtained earlier (this will also be in the dropdown but that dropdown will have many process IDs) 7. LLDB make take couple of seconds to attach to the process -8. When the debugger is attached to the process (you will notice the debugger panels get filled with extra info), enter `c`+Enter in the `PDB` prompt in your terminal - the execution should continue until the breakpoint in Rust code is hit. From this point it's a standard debugging procecess. +8. When the debugger is attached to the process (you will notice the debugger panels get filled with extra info), enter `c`+Enter in the `PDB` prompt in your terminal - the execution should continue until the breakpoint in Rust code is hit. From this point it's a standard debugging process. diff --git a/Cargo.toml b/Cargo.toml index 0fe711794c..a7c6c18624 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -26,45 +26,43 @@ debug = true debug = "line-tables-only" [workspace.dependencies] -delta_kernel = { version = "0.8.0", features = ["arrow_54"] } -#delta_kernel = { path = "../delta-kernel-rs/kernel", features = ["sync-engine"] } +delta_kernel = { version = "0.11.0", features = ["arrow-55", "internal-api"] } # arrow -arrow = { version = "54" } -arrow-arith = { version = "54" } -arrow-array = { version = "54", features = ["chrono-tz"] } -arrow-buffer = { version = "54" } -arrow-cast = { version = "54" } -arrow-ipc = { version = "54" } -arrow-json = { version = "54" } -arrow-ord = { version = "54" } -arrow-row = { version = "54" } -arrow-schema = { version = "54" } -arrow-select = { version = "54" } -object_store = { version = "0.11.2" } -parquet = { version = "54" } +arrow = { version = "=55.0.0" } +arrow-arith = { version = "=55.0.0" } +arrow-array = { version = "=55.0.0", features = ["chrono-tz"] } +arrow-buffer = { version = "=55.0.0" } +arrow-cast = { version = "=55.0.0" } +arrow-ipc = { version = "=55.0.0" } +arrow-json = { version = "=55.0.0" } +arrow-ord = { version = "=55.0.0" } +arrow-row = { version = "=55.0.0" } +arrow-schema = { version = "=55.0.0" } +arrow-select = { version = "=55.0.0" } +object_store = { version = "0.12.1" } +parquet = { version = "=55.0.0" } # datafusion -datafusion = "46" -datafusion-expr = "46" -datafusion-common = "46" -datafusion-ffi = "46" -datafusion-functions = "46" -datafusion-functions-aggregate = "46" -datafusion-physical-expr = "46" -datafusion-physical-plan = "46" -datafusion-proto = "46" -datafusion-sql = "46" +datafusion = "47.0.0" +datafusion-expr = "47.0.0" +datafusion-common = "47.0.0" +datafusion-ffi = "47.0.0" +datafusion-functions = "47.0.0" +datafusion-functions-aggregate = "47.0.0" +datafusion-physical-expr = "47.0.0" +datafusion-physical-plan = "47.0.0" +datafusion-proto = "47.0.0" +datafusion-sql = "47.0.0" # serde serde = { version = "1.0.194", features = ["derive"] } serde_json = "1" -strum = { version = "0.26"} - +strum = { version = "0.27" } # "stdlib" bytes = { version = "1" } -chrono = { version = "=0.4.39", default-features = false, features = ["clock"] } +chrono = { version = "0.4.40", default-features = false, features = ["clock"] } tracing = { version = "0.1", features = ["log"] } regex = { version = "1" } thiserror = { version = "2" } @@ -77,3 +75,21 @@ async-trait = { version = "0.1" } futures = { version = "0.3" } tokio = { version = "1" } num_cpus = { version = "1" } + +[workspace.metadata.typos] +default.extend-ignore-re = [ + # Custom ignore regex patterns: https://github.com/crate-ci/typos/blob/master/docs/reference.md#example-configurations + "(?s)//\\s*spellchecker:ignore-next-line[^\\n]*\\n[^\\n]*", + # Line block with # spellchecker: + "(?s)(#|//|<\\!--)\\s*spellchecker:off.*?\\n\\s*(#|//|<\\!--)\\s*spellchecker:on", + "(?Rm)^.*(#|//)\\s*spellchecker:disable-line$", + # workaround for: https://github.com/crate-ci/typos/issues/850 + "[0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}", +] + +[workspace.metadata.typos.default.extend-words] +arro = "arro" +Arro = "Arro" +arro3 = "arro3" +Arro3 = "Arro3" +AKS = "AKS" diff --git a/Makefile b/Makefile new file mode 100644 index 0000000000..84a7e79b4e --- /dev/null +++ b/Makefile @@ -0,0 +1,57 @@ +# +# This Makefile exists largely to help ensure that some of the common behaviors +# between CI and local development can be consistently replicated +# +# For the most part you should be able to rely on cargo for development. + +.DEFAULT_GOAL := help +DAT_VERSION := 0.0.3 +DEFAULT_FEATURES := "azure,datafusion,s3,gcs,glue,hdfs" + +# Disable full debug symbol generation to speed up CI build and keep memory down +export RUSTFLAGS:= -C debuginfo=line-tables-only +# Disable incremental builds by cargo for CI which should save disk space +# and hopefully avoid final link "No space left on device" +export CARGO_INCREMENTAL:=0 + +## begin dat related +#################### +.PHONY: setup-dat +setup-dat: dat/v$(DAT_VERSION) ## Download and setup the Delta Acceptance Tests (dat) + +dat: + mkdir -p dat + +dat/v$(DAT_VERSION): dat ## Download DAT test files into ./dat + curl -L --silent --output dat/deltalake-dat-v$(DAT_VERSION).tar.gz \ + https://github.com/delta-incubator/dat/releases/download/v$(DAT_VERSION)/deltalake-dat-v$(DAT_VERSION).tar.gz + tar --no-same-permissions -xzf dat/deltalake-dat-v$(DAT_VERSION).tar.gz + mv out dat/v$(DAT_VERSION) + rm dat/deltalake-dat-v$(DAT_VERSION).tar.gz + + +#################### +## end dat related + + +.PHONY: coverage +coverage: setup-dat ## Run Rust tests with code-coverage + cargo llvm-cov --features $(DEFAULT_FEATURES) --workspace \ + --codecov \ + --output-path codecov.json \ + -- --skip read_table_version_hdfs --skip test_read_tables_lakefs + +.PHONY: check +check: ## Run basic cargo formatting and other checks (no tests) + cargo fmt -- --check + + + +.PHONY: clean +clean: ## Remove temporary and downloaded artifacts + rm -rf dat + cargo clean + +.PHONY: help +help: ## Produce the helpful command listing + @grep -E '^[a-zA-Z_-]+:.*?## .*$$' $(MAKEFILE_LIST) | sort | awk 'BEGIN {FS = ":.*?## "}; {printf "\033[36m%-30s\033[0m %s\n", $$1, $$2}' diff --git a/README.md b/README.md index 92424b7bc5..834bcb2f92 100644 --- a/README.md +++ b/README.md @@ -190,7 +190,7 @@ of features outlined in the Delta [protocol][protocol] is also [tracked](#protoc | Version 2 | Column Mapping | | | Version 3 | Table Features (requires reader V7) | ![done] | -[datafusion]: https://github.com/apache/arrow-datafusion +[datafusion]: https://github.com/apache/datafusion [ballista]: https://github.com/apache/arrow-ballista [polars]: https://github.com/pola-rs/polars [open]: https://cdn.jsdelivr.net/gh/Readme-Workflows/Readme-Icons@main/icons/octicons/IssueNeutral.svg diff --git a/crates/aws/Cargo.toml b/crates/aws/Cargo.toml index 4102f7486f..7c13b31495 100644 --- a/crates/aws/Cargo.toml +++ b/crates/aws/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "deltalake-aws" -version = "0.8.0" +version = "0.9.1" authors.workspace = true keywords.workspace = true readme.workspace = true @@ -12,14 +12,8 @@ repository.workspace = true rust-version.workspace = true [dependencies] -deltalake-core = { version = "0.25.0", path = "../core", features = ["cloud"] } -aws-smithy-runtime-api = { version="1.7" } -aws-smithy-runtime = { version="1.7", optional = true} -aws-credential-types = { version="1.2", features = ["hardcoded-credentials"]} -aws-config = { version = "1.5", default-features = false, features = ["behavior-version-latest","rt-tokio", "credentials-process", "sso"] } -aws-sdk-dynamodb = {version = "1.45", default-features = false, features = ["behavior-version-latest", "rt-tokio"] } -aws-sdk-sts = {version = "1.42", default-features = false, features = ["behavior-version-latest", "rt-tokio"] } -maplit = "1" +# path dependencies +deltalake-core = { version = "0.26.0", path = "../core" } # workspace dependencies async-trait = { workspace = true } @@ -27,17 +21,37 @@ bytes = { workspace = true } chrono = { workspace = true } futures = { workspace = true } tracing = { workspace = true } -object_store = { workspace = true, features = ["aws"]} +object_store = { workspace = true, features = ["aws"] } +regex = { workspace = true } thiserror = { workspace = true } tokio = { workspace = true } -regex = { workspace = true } uuid = { workspace = true, features = ["serde", "v4"] } url = { workspace = true } -backon = { version = "1",default-features = false, features = [ "tokio-sleep" ] } + +# crates.io dependencies +aws-smithy-runtime-api = { version = "1.7" } +aws-smithy-runtime = { version = "1.7", optional = true } +aws-credential-types = { version = "1.2", features = ["hardcoded-credentials"] } +aws-config = { version = "1.5", default-features = false, features = [ + "behavior-version-latest", + "rt-tokio", + "credentials-process", + "sso", +] } +aws-sdk-dynamodb = { version = "1.45", default-features = false, features = [ + "behavior-version-latest", + "rt-tokio", +] } +aws-sdk-sts = { version = "1.42", default-features = false, features = [ + "behavior-version-latest", + "rt-tokio", +] } +backon = { version = "1", default-features = false, features = ["tokio-sleep"] } hyper-tls = { version = "0.5", optional = true } +maplit = "1" [dev-dependencies] -deltalake-core = { path = "../core", features = ["datafusion"] } +deltalake-core = { path = "../core" } chrono = { workspace = true } serial_test = "3" deltalake-test = { path = "../test" } @@ -51,7 +65,7 @@ integration_test = [] native-tls = [ "aws-config/client-hyper", "aws-smithy-runtime/connector-hyper-0-14-x", - "hyper-tls" + "hyper-tls", ] rustls = [ "aws-config/client-hyper", diff --git a/crates/aws/src/constants.rs b/crates/aws/src/constants.rs index 2fdd19dd5d..ecd37b15af 100644 --- a/crates/aws/src/constants.rs +++ b/crates/aws/src/constants.rs @@ -74,19 +74,19 @@ pub const AWS_S3_GET_INTERNAL_SERVER_ERROR_RETRIES: &str = /// The web identity token file to use when using a web identity provider. /// /// NOTE: web identity related options are set in the environment when -/// creating an instance of [crate::storage::s3::S3StorageOptions]. +/// creating an instance of [S3StorageOptions](crate::storage::S3StorageOptions). /// See also . pub const AWS_WEB_IDENTITY_TOKEN_FILE: &str = "AWS_WEB_IDENTITY_TOKEN_FILE"; /// The role name to use for web identity. /// /// NOTE: web identity related options are set in the environment when -/// creating an instance of [crate::storage::s3::S3StorageOptions]. +/// creating an instance of [S3StorageOptions](crate::storage::S3StorageOptions). /// See also . pub const AWS_ROLE_ARN: &str = "AWS_ROLE_ARN"; /// The role session name to use for web identity. /// /// NOTE: web identity related options are set in the environment when -/// creating an instance of [crate::storage::s3::S3StorageOptions]. +/// creating an instance of [S3StorageOptions](crate::storage::S3StorageOptions). /// See also . pub const AWS_ROLE_SESSION_NAME: &str = "AWS_ROLE_SESSION_NAME"; /// Allow http connections - mainly useful for integration tests @@ -109,7 +109,7 @@ pub const AWS_FORCE_CREDENTIAL_LOAD: &str = "AWS_FORCE_CREDENTIAL_LOAD"; /// The list of option keys owned by the S3 module. /// Option keys not contained in this list will be added to the `extra_opts` -/// field of [crate::storage::s3::S3StorageOptions]. +/// field of [S3StorageOptions](crate::storage::S3StorageOptions). #[allow(deprecated)] pub const S3_OPTS: &[&str] = &[ AWS_ENDPOINT_URL, diff --git a/crates/aws/src/credentials.rs b/crates/aws/src/credentials.rs index 6f138dc948..d2c3bec6c7 100644 --- a/crates/aws/src/credentials.rs +++ b/crates/aws/src/credentials.rs @@ -14,11 +14,10 @@ use aws_credential_types::provider::error::CredentialsError; use aws_credential_types::provider::{future, ProvideCredentials}; use aws_credential_types::Credentials; -use deltalake_core::storage::object_store::aws::{AmazonS3ConfigKey, AwsCredential}; -use deltalake_core::storage::object_store::{ +use deltalake_core::logstore::object_store::aws::{AmazonS3ConfigKey, AwsCredential}; +use deltalake_core::logstore::object_store::{ CredentialProvider, Error as ObjectStoreError, Result as ObjectStoreResult, }; -use deltalake_core::storage::StorageOptions; use deltalake_core::DeltaResult; use tokio::sync::Mutex; use tracing::log::*; @@ -117,7 +116,7 @@ const OPTS_PROVIDER: &str = "DeltaStorageOptionsProvider"; /// [aws_config::default_provider::credentials::DefaultCredentialsChain] #[derive(Clone, Debug)] pub(crate) struct OptionsCredentialsProvider { - options: StorageOptions, + options: HashMap, } impl OptionsCredentialsProvider { @@ -130,7 +129,7 @@ impl OptionsCredentialsProvider { // [object_store::aws::AmazonS3ConfigKey] supports a couple different variants for key // names. let config_keys: HashMap = - HashMap::from_iter(self.options.0.iter().filter_map(|(k, v)| { + HashMap::from_iter(self.options.iter().filter_map(|(k, v)| { match AmazonS3ConfigKey::from_str(&k.to_lowercase()) { Ok(k) => Some((k, v.into())), Err(_) => None, @@ -171,7 +170,7 @@ mod options_tests { #[test] fn test_empty_options_error() { - let options = StorageOptions::default(); + let options = HashMap::default(); let provider = OptionsCredentialsProvider { options }; let result = provider.credentials(); assert!( @@ -182,11 +181,10 @@ mod options_tests { #[test] fn test_uppercase_options_resolve() { - let mash = hashmap! { + let options = hashmap! { "AWS_ACCESS_KEY_ID".into() => "key".into(), "AWS_SECRET_ACCESS_KEY".into() => "secret".into(), }; - let options = StorageOptions(mash); let provider = OptionsCredentialsProvider { options }; let result = provider.credentials(); assert!(result.is_ok(), "StorageOptions with at least AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY should resolve"); @@ -197,11 +195,10 @@ mod options_tests { #[test] fn test_lowercase_options_resolve() { - let mash = hashmap! { + let options = hashmap! { "aws_access_key_id".into() => "key".into(), "aws_secret_access_key".into() => "secret".into(), }; - let options = StorageOptions(mash); let provider = OptionsCredentialsProvider { options }; let result = provider.credentials(); assert!(result.is_ok(), "StorageOptions with at least AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY should resolve"); @@ -219,13 +216,12 @@ fn assume_role_session_name() -> String { } /// Return the configured IAM role ARN or whatever is defined in the environment -fn assume_role_arn(options: &StorageOptions) -> Option { +fn assume_role_arn(options: &HashMap) -> Option { options - .0 .get(constants::AWS_IAM_ROLE_ARN) .or( #[allow(deprecated)] - options.0.get(constants::AWS_S3_ASSUME_ROLE_ARN), + options.get(constants::AWS_S3_ASSUME_ROLE_ARN), ) .or(std::env::var_os(constants::AWS_IAM_ROLE_ARN) .map(|o| { @@ -247,31 +243,29 @@ fn assume_role_arn(options: &StorageOptions) -> Option { } /// Return the configured IAM assume role session name or provide a unique one -fn assume_session_name(options: &StorageOptions) -> String { +fn assume_session_name(options: &HashMap) -> String { let assume_session = options - .0 .get(constants::AWS_IAM_ROLE_SESSION_NAME) .or( #[allow(deprecated)] - options.0.get(constants::AWS_S3_ROLE_SESSION_NAME), + options.get(constants::AWS_S3_ROLE_SESSION_NAME), ) .cloned(); - assume_session.unwrap_or_else(assume_role_session_name) } /// Take a set of [StorageOptions] and produce an appropriate AWS SDK [SdkConfig] -/// for use with various AWS SDK APIs, such as in our [crate::logstore::S3DynamoDbLogStore] -pub async fn resolve_credentials(options: StorageOptions) -> DeltaResult { +/// for use with various AWS SDK APIs, such as in our [S3DynamoDbLogStore](crate::logstore::S3DynamoDbLogStore) +pub async fn resolve_credentials(options: &HashMap) -> DeltaResult { let default_provider = DefaultCredentialsChain::builder().build().await; - let credentials_provider = match assume_role_arn(&options) { + let credentials_provider = match assume_role_arn(options) { Some(arn) => { debug!("Configuring AssumeRoleProvider with role arn: {arn}"); CredentialsProviderChain::first_try( "AssumeRoleProvider", AssumeRoleProvider::builder(arn) - .session_name(assume_session_name(&options)) + .session_name(assume_session_name(options)) .build() .await, ) @@ -308,12 +302,12 @@ mod tests { #[tokio::test] #[serial] async fn test_options_credentials_provider() { - let options = StorageOptions(hashmap! { + let options = hashmap! { constants::AWS_ACCESS_KEY_ID.to_string() => "test_id".to_string(), constants::AWS_SECRET_ACCESS_KEY.to_string() => "test_secret".to_string(), - }); + }; - let config = resolve_credentials(options).await; + let config = resolve_credentials(&options).await; assert!(config.is_ok(), "{config:?}"); let config = config.unwrap(); @@ -340,13 +334,13 @@ mod tests { #[tokio::test] #[serial] async fn test_options_credentials_provider_session_token() { - let options = StorageOptions(hashmap! { + let options = hashmap! { constants::AWS_ACCESS_KEY_ID.to_string() => "test_id".to_string(), constants::AWS_SECRET_ACCESS_KEY.to_string() => "test_secret".to_string(), constants::AWS_SESSION_TOKEN.to_string() => "test_token".to_string(), - }); + }; - let config = resolve_credentials(options) + let config = resolve_credentials(&options) .await .expect("Failed to resolve_credentials"); @@ -368,11 +362,11 @@ mod tests { #[tokio::test] #[serial] async fn test_object_store_credential_provider() -> DeltaResult<()> { - let options = StorageOptions(hashmap! { + let options = hashmap! { constants::AWS_ACCESS_KEY_ID.to_string() => "test_id".to_string(), constants::AWS_SECRET_ACCESS_KEY.to_string() => "test_secret".to_string(), - }); - let sdk_config = resolve_credentials(options) + }; + let sdk_config = resolve_credentials(&options) .await .expect("Failed to resolve credentials for the test"); let provider = AWSForObjectStore::new(sdk_config); @@ -392,11 +386,11 @@ mod tests { #[tokio::test] #[serial] async fn test_object_store_credential_provider_consistency() -> DeltaResult<()> { - let options = StorageOptions(hashmap! { + let options = hashmap! { constants::AWS_ACCESS_KEY_ID.to_string() => "test_id".to_string(), constants::AWS_SECRET_ACCESS_KEY.to_string() => "test_secret".to_string(), - }); - let sdk_config = resolve_credentials(options) + }; + let sdk_config = resolve_credentials(&options) .await .expect("Failed to resolve credentijals for the test"); let provider = AWSForObjectStore::new(sdk_config); diff --git a/crates/aws/src/lib.rs b/crates/aws/src/lib.rs index 76c701c474..de1d61f4ab 100644 --- a/crates/aws/src/lib.rs +++ b/crates/aws/src/lib.rs @@ -1,7 +1,7 @@ //! AWS S3 and similar tooling for delta-rs //! -//! This module also contains the [S3DynamoDbLogStore] implementation for concurrent writer support -//! with AWS S3 specifically. +//! This module also contains the [S3DynamoDbLogStore](crate::logstore::S3DynamoDbLogStore) +//! implementation for concurrent writer support with AWS S3 specifically. pub mod constants; mod credentials; @@ -10,6 +10,7 @@ pub mod logstore; #[cfg(feature = "native-tls")] mod native; pub mod storage; + use aws_config::Region; use aws_config::SdkConfig; pub use aws_credential_types::provider::SharedCredentialsProvider; @@ -25,9 +26,11 @@ use aws_sdk_dynamodb::{ }, Client, }; -use deltalake_core::logstore::{default_logstore, logstores, LogStore, LogStoreFactory}; -use deltalake_core::storage::object_store::aws::AmazonS3ConfigKey; -use deltalake_core::storage::{factories, url_prefix_handler, ObjectStoreRef, StorageOptions}; +use deltalake_core::logstore::object_store::aws::AmazonS3ConfigKey; +use deltalake_core::logstore::{ + default_logstore, logstore_factories, object_store_factories, LogStore, LogStoreFactory, + ObjectStoreRef, StorageConfig, +}; use deltalake_core::{DeltaResult, Path}; use errors::{DynamoDbConfigError, LockClientError}; use regex::Regex; @@ -51,13 +54,13 @@ impl S3StorageOptionsConversion for S3LogStoreFactory {} impl LogStoreFactory for S3LogStoreFactory { fn with_options( &self, - store: ObjectStoreRef, + prefixed_store: ObjectStoreRef, + root_store: ObjectStoreRef, location: &Url, - options: &StorageOptions, + options: &StorageConfig, ) -> DeltaResult> { - let store = url_prefix_handler(store, Path::parse(location.path())?); - let options = self.with_env_s3(options); - if options.0.keys().any(|key| { + let s3_options = self.with_env_s3(&options.raw.clone()); + if s3_options.keys().any(|key| { let key = key.to_ascii_lowercase(); [ AmazonS3ConfigKey::CopyIfNotExists.as_ref(), @@ -67,31 +70,44 @@ impl LogStoreFactory for S3LogStoreFactory { }) { debug!("S3LogStoreFactory has been asked to create a LogStore where the underlying store has copy-if-not-exists enabled - no locking provider required"); warn!("Most S3 object store support conditional put, remove copy_if_not_exists parameter to use a more performant conditional put."); - return Ok(logstore::default_s3_logstore(store, location, &options)); + return Ok(logstore::default_s3_logstore( + prefixed_store, + root_store, + location, + options, + )); } - let s3_options = S3StorageOptions::from_map(&options.0)?; + let s3_options = S3StorageOptions::from_map(&s3_options)?; if s3_options.locking_provider.as_deref() == Some("dynamodb") { debug!("S3LogStoreFactory has been asked to create a LogStore with the dynamodb locking provider"); return Ok(Arc::new(logstore::S3DynamoDbLogStore::try_new( location.clone(), - options.clone(), + options, &s3_options, - store, + prefixed_store, + root_store, )?)); } - Ok(default_logstore(store, location, &options)) + Ok(default_logstore( + prefixed_store, + root_store, + location, + options, + )) } } -/// Register an [ObjectStoreFactory] for common S3 [Url] schemes +/// Register an [ObjectStoreFactory] for common S3 url schemes. +/// +/// [ObjectStoreFactory]: deltalake_core::logstore::ObjectStoreFactory pub fn register_handlers(_additional_prefixes: Option) { let object_stores = Arc::new(S3ObjectStoreFactory::default()); let log_stores = Arc::new(S3LogStoreFactory::default()); for scheme in ["s3", "s3a"].iter() { let url = Url::parse(&format!("{scheme}://")).unwrap(); - factories().insert(url.clone(), object_stores.clone()); - logstores().insert(url.clone(), log_stores.clone()); + object_store_factories().insert(url.clone(), object_stores.clone()); + logstore_factories().insert(url.clone(), log_stores.clone()); } } @@ -757,13 +773,13 @@ mod tests { #[serial] fn test_logstore_factory_default() { let factory = S3LogStoreFactory::default(); - let store = InMemory::new(); + let store = Arc::new(InMemory::new()); let url = Url::parse("s3://test-bucket").unwrap(); unsafe { std::env::remove_var(crate::constants::AWS_S3_LOCKING_PROVIDER); } let logstore = factory - .with_options(Arc::new(store), &url, &StorageOptions::from(HashMap::new())) + .with_options(store.clone(), store, &url, &Default::default()) .unwrap(); assert_eq!(logstore.name(), "DefaultLogStore"); } @@ -772,14 +788,14 @@ mod tests { #[serial] fn test_logstore_factory_with_locking_provider() { let factory = S3LogStoreFactory::default(); - let store = InMemory::new(); + let store = Arc::new(InMemory::new()); let url = Url::parse("s3://test-bucket").unwrap(); unsafe { std::env::set_var(crate::constants::AWS_S3_LOCKING_PROVIDER, "dynamodb"); } let logstore = factory - .with_options(Arc::new(store), &url, &StorageOptions::from(HashMap::new())) + .with_options(store.clone(), store, &url, &Default::default()) .unwrap(); assert_eq!(logstore.name(), "S3DynamoDbLogStore"); } diff --git a/crates/aws/src/logstore/default_logstore.rs b/crates/aws/src/logstore/default_logstore.rs index 7677010f15..75c6d75a22 100644 --- a/crates/aws/src/logstore/default_logstore.rs +++ b/crates/aws/src/logstore/default_logstore.rs @@ -5,9 +5,7 @@ use std::sync::Arc; use bytes::Bytes; use deltalake_core::logstore::*; use deltalake_core::{ - operations::transaction::TransactionError, - storage::{ObjectStoreRef, StorageOptions}, - DeltaResult, + kernel::transaction::TransactionError, logstore::ObjectStoreRef, DeltaResult, }; use object_store::{Error as ObjectStoreError, ObjectStore}; use url::Url; @@ -16,11 +14,13 @@ use uuid::Uuid; /// Return the [S3LogStore] implementation with the provided configuration options pub fn default_s3_logstore( store: ObjectStoreRef, + root_store: ObjectStoreRef, location: &Url, - options: &StorageOptions, + options: &StorageConfig, ) -> Arc { Arc::new(S3LogStore::new( store, + root_store, LogStoreConfig { location: location.clone(), options: options.clone(), @@ -31,7 +31,8 @@ pub fn default_s3_logstore( /// Default [`LogStore`] implementation #[derive(Debug, Clone)] pub struct S3LogStore { - pub(crate) storage: ObjectStoreRef, + prefixed_store: ObjectStoreRef, + root_store: ObjectStoreRef, config: LogStoreConfig, } @@ -40,10 +41,21 @@ impl S3LogStore { /// /// # Arguments /// - /// * `storage` - A shared reference to an [`object_store::ObjectStore`] with "/" pointing at delta table root (i.e. where `_delta_log` is located). + /// * `prefixed_store` - A shared reference to an [`object_store::ObjectStore`] + /// with "/" pointing at delta table root (i.e. where `_delta_log` is located). + /// * `root_store` - A shared reference to an [`object_store::ObjectStore`] with "/" + /// pointing at root of the storage system. /// * `location` - A url corresponding to the storage location of `storage`. - pub fn new(storage: ObjectStoreRef, config: LogStoreConfig) -> Self { - Self { storage, config } + pub fn new( + prefixed_store: ObjectStoreRef, + root_store: ObjectStoreRef, + config: LogStoreConfig, + ) -> Self { + Self { + prefixed_store, + root_store, + config, + } } } @@ -106,12 +118,12 @@ impl LogStore for S3LogStore { get_latest_version(self, current_version).await } - async fn get_earliest_version(&self, current_version: i64) -> DeltaResult { - get_earliest_version(self, current_version).await + fn object_store(&self, _operation_id: Option) -> Arc { + self.prefixed_store.clone() } - fn object_store(&self, _operation_id: Option) -> Arc { - self.storage.clone() + fn root_object_store(&self, _operation_id: Option) -> Arc { + self.root_store.clone() } fn config(&self) -> &LogStoreConfig { diff --git a/crates/aws/src/logstore/dynamodb_logstore.rs b/crates/aws/src/logstore/dynamodb_logstore.rs index f5680e091c..c8b3cb9c16 100644 --- a/crates/aws/src/logstore/dynamodb_logstore.rs +++ b/crates/aws/src/logstore/dynamodb_logstore.rs @@ -14,9 +14,7 @@ use url::Url; use deltalake_core::logstore::*; use deltalake_core::{ - operations::transaction::TransactionError, - storage::{ObjectStoreRef, StorageOptions}, - DeltaResult, DeltaTableError, + kernel::transaction::TransactionError, logstore::ObjectStoreRef, DeltaResult, DeltaTableError, }; use uuid::Uuid; @@ -25,7 +23,8 @@ const MAX_REPAIR_RETRIES: i64 = 3; /// [`LogStore`] implementation backed by DynamoDb pub struct S3DynamoDbLogStore { - pub(crate) storage: ObjectStoreRef, + prefixed_store: ObjectStoreRef, + root_store: ObjectStoreRef, lock_client: DynamoDbLockClient, config: LogStoreConfig, table_path: String, @@ -41,9 +40,10 @@ impl S3DynamoDbLogStore { /// Create log store pub fn try_new( location: Url, - options: impl Into + Clone, + options: &StorageConfig, s3_options: &S3StorageOptions, - object_store: ObjectStoreRef, + prefixed_store: ObjectStoreRef, + root_store: ObjectStoreRef, ) -> DeltaResult { let lock_client = DynamoDbLockClient::try_new( &s3_options.sdk_config.clone().unwrap(), @@ -73,11 +73,12 @@ impl S3DynamoDbLogStore { })?; let table_path = to_uri(&location, &Path::from("")); Ok(Self { - storage: object_store, + prefixed_store, + root_store, lock_client, config: LogStoreConfig { location, - options: options.into(), + options: options.clone(), }, table_path, }) @@ -309,12 +310,12 @@ impl LogStore for S3DynamoDbLogStore { } } - async fn get_earliest_version(&self, current_version: i64) -> DeltaResult { - get_earliest_version(self, current_version).await + fn object_store(&self, _operation_id: Option) -> ObjectStoreRef { + self.prefixed_store.clone() } - fn object_store(&self, _operation_id: Option) -> ObjectStoreRef { - self.storage.clone() + fn root_object_store(&self, _operation_id: Option) -> ObjectStoreRef { + self.root_store.clone() } fn config(&self) -> &LogStoreConfig { diff --git a/crates/aws/src/native.rs b/crates/aws/src/native.rs index c647194eb7..4c4f2d41aa 100644 --- a/crates/aws/src/native.rs +++ b/crates/aws/src/native.rs @@ -1,12 +1,12 @@ use aws_sdk_sts::config::SharedHttpClient; use aws_smithy_runtime::client::http::hyper_014::HyperClientBuilder; +#[allow(dead_code)] pub fn use_native_tls_client(allow_http: bool) -> SharedHttpClient { let mut tls_connector = hyper_tls::HttpsConnector::new(); if allow_http { tls_connector.https_only(false); } - let client = HyperClientBuilder::new().build(tls_connector); - client + HyperClientBuilder::new().build(tls_connector) } diff --git a/crates/aws/src/storage.rs b/crates/aws/src/storage.rs index a1335afca7..8d167780f1 100644 --- a/crates/aws/src/storage.rs +++ b/crates/aws/src/storage.rs @@ -1,32 +1,32 @@ //! AWS S3 storage backend. +use std::collections::HashMap; +use std::fmt::Debug; +use std::ops::Range; +use std::str::FromStr; +use std::sync::Arc; +use std::time::Duration; use aws_config::{Region, SdkConfig}; use bytes::Bytes; -use deltalake_core::storage::object_store::aws::{AmazonS3Builder, AmazonS3ConfigKey}; -use deltalake_core::storage::object_store::{ +use deltalake_core::logstore::object_store::aws::{AmazonS3Builder, AmazonS3ConfigKey}; +use deltalake_core::logstore::object_store::{ GetOptions, GetResult, ListResult, MultipartUpload, ObjectMeta, ObjectStore, ObjectStoreScheme, PutMultipartOpts, PutOptions, PutPayload, PutResult, Result as ObjectStoreResult, }; -use deltalake_core::storage::{ - limit_store_handler, str_is_truthy, ObjectStoreFactory, ObjectStoreRef, RetryConfigParse, - StorageOptions, -}; +use deltalake_core::logstore::{config::str_is_truthy, ObjectStoreFactory, ObjectStoreRef}; use deltalake_core::{DeltaResult, DeltaTableError, ObjectStoreError, Path}; use futures::stream::BoxStream; use futures::Future; -use std::collections::HashMap; -use std::fmt::Debug; -use std::ops::Range; -use std::str::FromStr; -use std::sync::Arc; -use std::time::Duration; +use object_store::aws::AmazonS3; +use object_store::client::SpawnedReqwestConnector; +use object_store::RetryConfig; +use tokio::runtime::Handle; use tracing::log::*; use url::Url; use crate::constants; +use crate::credentials::AWSForObjectStore; use crate::errors::DynamoDbConfigError; -#[cfg(feature = "native-tls")] -use crate::native; const STORE_NAME: &str = "DeltaS3ObjectStore"; @@ -35,44 +35,44 @@ pub struct S3ObjectStoreFactory {} impl S3StorageOptionsConversion for S3ObjectStoreFactory {} -impl RetryConfigParse for S3ObjectStoreFactory {} - impl ObjectStoreFactory for S3ObjectStoreFactory { fn parse_url_opts( &self, url: &Url, - storage_options: &StorageOptions, + storage_options: &HashMap, + retry: &RetryConfig, + handle: Option, ) -> DeltaResult<(ObjectStoreRef, Path)> { let options = self.with_env_s3(storage_options); // All S3-likes should start their builder the same way - let mut builder = AmazonS3Builder::new().with_url(url.to_string()); + let mut builder = AmazonS3Builder::new() + .with_url(url.to_string()) + .with_retry(retry.clone()); + + if let Some(handle) = handle { + builder = builder.with_http_connector(SpawnedReqwestConnector::new(handle)); + } - for (key, value) in options.0.iter() { + for (key, value) in options.iter() { if let Ok(key) = AmazonS3ConfigKey::from_str(&key.to_ascii_lowercase()) { builder = builder.with_config(key, value.clone()); } } + let s3_options = S3StorageOptions::from_map(&options)?; + if let Some(ref sdk_config) = s3_options.sdk_config { + builder = + builder.with_credentials(Arc::new(AWSForObjectStore::new(sdk_config.clone()))); + } + let (_, path) = ObjectStoreScheme::parse(url).map_err(|e| DeltaTableError::GenericError { source: Box::new(e), })?; let prefix = Path::parse(path)?; - let s3_options: S3StorageOptions = S3StorageOptions::from_map(&options.0)?; - - if let Some(ref sdk_config) = s3_options.sdk_config { - builder = builder.with_credentials(Arc::new( - crate::credentials::AWSForObjectStore::new(sdk_config.clone()), - )); - } - - let inner = builder - .with_retry(self.parse_retry_config(&options)?) - .build()?; - - let store = aws_storage_handler(limit_store_handler(inner, &options), &s3_options)?; + let store = aws_storage_handler(builder.build()?, &s3_options)?; debug!("Initialized the object store: {store:?}"); Ok((store, prefix)) @@ -80,7 +80,7 @@ impl ObjectStoreFactory for S3ObjectStoreFactory { } fn aws_storage_handler( - store: ObjectStoreRef, + store: AmazonS3, s3_options: &S3StorageOptions, ) -> DeltaResult { // Nearly all S3 Object stores support conditional put, so we change the default to always returning an S3 Object store @@ -88,13 +88,13 @@ fn aws_storage_handler( if s3_options.locking_provider.as_deref() == Some("dynamodb") || s3_options.allow_unsafe_rename { let store = S3StorageBackend::try_new( - store, + Arc::new(store), Some("dynamodb") == s3_options.locking_provider.as_deref() || s3_options.allow_unsafe_rename, )?; Ok(Arc::new(store)) } else { - Ok(store) + Ok(Arc::new(store)) } } @@ -102,22 +102,22 @@ fn aws_storage_handler( // // This function will return true in the default case since it's most likely that the absence of // options will mean default/S3 configuration -fn is_aws(options: &StorageOptions) -> bool { +fn is_aws(options: &HashMap) -> bool { // Checks storage option first then env var for existence of aws force credential load // .from_s3_env never inserts these into the options because they are delta-rs specific - if str_option(&options.0, constants::AWS_FORCE_CREDENTIAL_LOAD).is_some() { + if str_option(options, constants::AWS_FORCE_CREDENTIAL_LOAD).is_some() { return true; } // Checks storage option first then env var for existence of locking provider // .from_s3_env never inserts these into the options because they are delta-rs specific - if str_option(&options.0, constants::AWS_S3_LOCKING_PROVIDER).is_some() { + if str_option(options, constants::AWS_S3_LOCKING_PROVIDER).is_some() { return true; } // Options at this stage should only contain 'aws_endpoint' in lowercase // due to with_env_s3 - !(options.0.contains_key("aws_endpoint") || options.0.contains_key(constants::AWS_ENDPOINT_URL)) + !(options.contains_key("aws_endpoint") || options.contains_key(constants::AWS_ENDPOINT_URL)) } /// Options used to configure the [S3StorageBackend]. @@ -161,7 +161,7 @@ impl PartialEq for S3StorageOptions { } impl S3StorageOptions { - /// Creates an instance of S3StorageOptions from the given HashMap. + /// Creates an instance of [`S3StorageOptions`] from the given HashMap. pub fn from_map(options: &HashMap) -> DeltaResult { let extra_opts: HashMap = options .iter() @@ -198,14 +198,12 @@ impl S3StorageOptions { .map(|val| str_is_truthy(&val)) .unwrap_or(false); - let storage_options = StorageOptions(options.clone()); - - let sdk_config = match is_aws(&storage_options) { + let sdk_config = match is_aws(options) { false => None, true => { debug!("Detected AWS S3 Storage options, resolving AWS credentials"); Some(execute_sdk_future( - crate::credentials::resolve_credentials(storage_options.clone()), + crate::credentials::resolve_credentials(options), )??) } }; @@ -368,7 +366,7 @@ impl ObjectStore for S3StorageBackend { self.inner.get_opts(location, options).await } - async fn get_range(&self, location: &Path, range: Range) -> ObjectStoreResult { + async fn get_range(&self, location: &Path, range: Range) -> ObjectStoreResult { self.inner.get_range(location, range).await } @@ -380,7 +378,7 @@ impl ObjectStore for S3StorageBackend { self.inner.delete(location).await } - fn list(&self, prefix: Option<&Path>) -> BoxStream<'_, ObjectStoreResult> { + fn list(&self, prefix: Option<&Path>) -> BoxStream<'static, ObjectStoreResult> { self.inner.list(prefix) } @@ -388,7 +386,7 @@ impl ObjectStore for S3StorageBackend { &self, prefix: Option<&Path>, offset: &Path, - ) -> BoxStream<'_, ObjectStoreResult> { + ) -> BoxStream<'static, ObjectStoreResult> { self.inner.list_with_offset(prefix, offset) } @@ -416,7 +414,7 @@ impl ObjectStore for S3StorageBackend { } } -/// Storage option keys to use when creating [crate::storage::s3::S3StorageOptions]. +/// Storage option keys to use when creating [`S3StorageOptions`]. /// /// The same key should be used whether passing a key in the hashmap or setting it as an environment variable. /// Provided keys may include configuration for the S3 backend and also the optional DynamoDb lock used for atomic rename. @@ -441,27 +439,23 @@ pub(crate) fn str_option(map: &HashMap, key: &str) -> Option StorageOptions { - let mut options = StorageOptions( - options - .0 - .clone() - .into_iter() - .map(|(k, v)| { - if let Ok(config_key) = AmazonS3ConfigKey::from_str(&k.to_ascii_lowercase()) { - (config_key.as_ref().to_string(), v) - } else { - (k, v) - } - }) - .collect(), - ); + fn with_env_s3(&self, options: &HashMap) -> HashMap { + let mut options: HashMap = options + .clone() + .into_iter() + .map(|(k, v)| { + if let Ok(config_key) = AmazonS3ConfigKey::from_str(&k.to_ascii_lowercase()) { + (config_key.as_ref().to_string(), v) + } else { + (k, v) + } + }) + .collect(); for (os_key, os_value) in std::env::vars_os() { if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { if let Ok(config_key) = AmazonS3ConfigKey::from_str(&key.to_ascii_lowercase()) { options - .0 .entry(config_key.as_ref().to_string()) .or_insert(value.to_string()); } @@ -472,7 +466,7 @@ pub(crate) trait S3StorageOptionsConversion { // set this behaviour. We will however assume, when a locking provider/copy-if-not-exists keys are not provided // that PutIfAbsent is supported. // With conditional put in S3-like API we can use the deltalake default logstore which use PutIfAbsent - if !options.0.keys().any(|key| { + if !options.keys().any(|key| { let key = key.to_ascii_lowercase(); [ AmazonS3ConfigKey::ConditionalPut.as_ref(), @@ -480,7 +474,7 @@ pub(crate) trait S3StorageOptionsConversion { ] .contains(&key.as_str()) }) { - options.0.insert("conditional_put".into(), "etag".into()); + options.insert("conditional_put".into(), "etag".into()); } options } @@ -793,13 +787,12 @@ mod tests { std::env::set_var(constants::AWS_ENDPOINT_URL, "env_key"); std::env::set_var(constants::AWS_SECRET_ACCESS_KEY, "env_key"); std::env::set_var(constants::AWS_REGION, "env_key"); - let combined_options = - S3ObjectStoreFactory {}.with_env_s3(&StorageOptions(raw_options)); + let combined_options = S3ObjectStoreFactory {}.with_env_s3(&raw_options); // Four and then the conditional_put built-in - assert_eq!(combined_options.0.len(), 5); + assert_eq!(combined_options.len(), 5); - for (key, v) in combined_options.0 { + for (key, v) in combined_options { if key != "conditional_put" { assert_eq!(v, "env_key"); } @@ -823,10 +816,9 @@ mod tests { std::env::set_var("aws_secret_access_key", "env_key"); std::env::set_var("aws_region", "env_key"); - let combined_options = - S3ObjectStoreFactory {}.with_env_s3(&StorageOptions(raw_options)); + let combined_options = S3ObjectStoreFactory {}.with_env_s3(&raw_options); - for (key, v) in combined_options.0 { + for (key, v) in combined_options { if key != "conditional_put" { assert_eq!(v, "options_key"); } @@ -838,26 +830,23 @@ mod tests { #[serial] fn test_is_aws() { clear_env_of_aws_keys(); - let options = StorageOptions::default(); + let options = HashMap::default(); assert!(is_aws(&options)); let minio: HashMap = hashmap! { constants::AWS_ENDPOINT_URL.to_string() => "http://minio:8080".to_string(), }; - let options = StorageOptions::from(minio); - assert!(!is_aws(&options)); + assert!(!is_aws(&minio)); let minio: HashMap = hashmap! { "aws_endpoint".to_string() => "http://minio:8080".to_string(), }; - let options = StorageOptions::from(minio); - assert!(!is_aws(&options)); + assert!(!is_aws(&minio)); let localstack: HashMap = hashmap! { constants::AWS_FORCE_CREDENTIAL_LOAD.to_string() => "true".to_string(), "aws_endpoint".to_string() => "http://minio:8080".to_string(), }; - let options = StorageOptions::from(localstack); - assert!(is_aws(&options)); + assert!(is_aws(&localstack)); } } diff --git a/crates/aws/tests/integration_s3_dynamodb.rs b/crates/aws/tests/integration_s3_dynamodb.rs index 1585c912c0..dad3aa2bb9 100644 --- a/crates/aws/tests/integration_s3_dynamodb.rs +++ b/crates/aws/tests/integration_s3_dynamodb.rs @@ -10,13 +10,12 @@ use aws_sdk_dynamodb::types::BillingMode; use deltalake_aws::logstore::{RepairLogEntryResult, S3DynamoDbLogStore}; use deltalake_aws::storage::S3StorageOptions; use deltalake_aws::{CommitEntry, DynamoDbConfig, DynamoDbLockClient}; +use deltalake_core::kernel::transaction::CommitBuilder; use deltalake_core::kernel::{Action, Add, DataType, PrimitiveType, StructField, StructType}; +use deltalake_core::logstore::{commit_uri_from_version, StorageConfig}; use deltalake_core::logstore::{logstore_for, CommitOrBytes, LogStore}; use deltalake_core::operations::create::CreateBuilder; -use deltalake_core::operations::transaction::CommitBuilder; use deltalake_core::protocol::{DeltaOperation, SaveMode}; -use deltalake_core::storage::commit_uri_from_version; -use deltalake_core::storage::StorageOptions; use deltalake_core::table::builder::ensure_table_uri; use deltalake_core::{DeltaOps, DeltaTable, DeltaTableBuilder, ObjectStoreError}; use deltalake_test::utils::*; @@ -65,13 +64,10 @@ fn client_configs_via_env_variables() -> TestResult<()> { deltalake_aws::constants::MAX_ELAPSED_REQUEST_TIME_KEY_NAME, "64", ); - std::env::set_var( - deltalake_aws::constants::LOCK_TABLE_KEY_NAME, - "some_table".to_owned(), - ); + std::env::set_var(deltalake_aws::constants::LOCK_TABLE_KEY_NAME, "some_table"); std::env::set_var( deltalake_aws::constants::BILLING_MODE_KEY_NAME, - "PAY_PER_REQUEST".to_owned(), + "PAY_PER_REQUEST", ); let client = make_client()?; let config = client.get_dynamodb_config(); @@ -111,7 +107,8 @@ async fn test_create_s3_table() -> TestResult<()> { deltalake_aws::constants::AWS_FORCE_CREDENTIAL_LOAD.into() => "true".into(), deltalake_aws::constants::AWS_ENDPOINT_URL.into() => "http://localhost:4566".into(), }; - let log_store = logstore_for(Url::parse(&table_uri)?, storage_options, None)?; + let storage_config = StorageConfig::parse_options(storage_options)?; + let log_store = logstore_for(Url::parse(&table_uri)?, storage_config)?; let payload = PutPayload::from_static(b"test-drivin"); let _put = log_store @@ -163,12 +160,13 @@ async fn test_repair_commit_entry() -> TestResult<()> { let context = IntegrationContext::new(Box::new(S3Integration::default()))?; let client = make_client()?; let table = prepare_table(&context, "repair_needed").await?; - let options: StorageOptions = OPTIONS.clone().into(); + let options: StorageConfig = OPTIONS.clone().into_iter().collect(); let log_store: S3DynamoDbLogStore = S3DynamoDbLogStore::try_new( ensure_table_uri(table.table_uri())?, - options.clone(), + &options, &S3_OPTIONS, - std::sync::Arc::new(table.object_store()), + table.log_store().object_store(None), + table.log_store().root_object_store(None), )?; // create an incomplete log entry, commit file not yet moved from its temporary location @@ -214,7 +212,7 @@ async fn test_repair_on_update() -> TestResult<()> { let _entry = create_incomplete_commit_entry(&table, 1, "unfinished_commit").await?; table.update().await?; // table update should find and update to newest, incomplete commit entry - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); validate_lock_table_state(&table, 1).await?; Ok(()) } @@ -227,7 +225,7 @@ async fn test_repair_on_load() -> TestResult<()> { let _entry = create_incomplete_commit_entry(&table, 1, "unfinished_commit").await?; table.load_version(1).await?; // table should fix the broken entry while loading a specific version - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); validate_lock_table_state(&table, 1).await?; Ok(()) } @@ -238,12 +236,13 @@ async fn test_abort_commit_entry() -> TestResult<()> { let context = IntegrationContext::new(Box::new(S3Integration::default()))?; let client = make_client()?; let table = prepare_table(&context, "abort_entry").await?; - let options: StorageOptions = OPTIONS.clone().into(); + let options: StorageConfig = OPTIONS.clone().into_iter().collect(); let log_store: S3DynamoDbLogStore = S3DynamoDbLogStore::try_new( ensure_table_uri(table.table_uri())?, - options.clone(), + &options, &S3_OPTIONS, - std::sync::Arc::new(table.object_store()), + table.log_store().object_store(None), + table.log_store().root_object_store(None), )?; let entry = create_incomplete_commit_entry(&table, 1, "unfinished_commit").await?; @@ -285,12 +284,13 @@ async fn test_abort_commit_entry_fail_to_delete_entry() -> TestResult<()> { let context = IntegrationContext::new(Box::new(S3Integration::default()))?; let client = make_client()?; let table = prepare_table(&context, "abort_entry_fail").await?; - let options: StorageOptions = OPTIONS.clone().into(); + let options: StorageConfig = OPTIONS.clone().into_iter().collect(); let log_store: S3DynamoDbLogStore = S3DynamoDbLogStore::try_new( ensure_table_uri(table.table_uri())?, - options.clone(), + &options, &S3_OPTIONS, - std::sync::Arc::new(table.object_store()), + table.log_store().object_store(None), + table.log_store().root_object_store(None), )?; let entry = create_incomplete_commit_entry(&table, 1, "finished_commit").await?; @@ -301,16 +301,14 @@ async fn test_abort_commit_entry_fail_to_delete_entry() -> TestResult<()> { .await?; // Abort will fail since we marked the entry as complete - assert!(matches!( - log_store - .abort_commit_entry( - entry.version, - CommitOrBytes::TmpCommit(entry.temp_path.clone()), - Uuid::new_v4(), - ) - .await, - Err(_), - )); + assert!(log_store + .abort_commit_entry( + entry.version, + CommitOrBytes::TmpCommit(entry.temp_path.clone()), + Uuid::new_v4(), + ) + .await + .is_err()); // Check temp commit file still exists assert!(log_store @@ -437,7 +435,6 @@ fn add_action(name: &str) -> Action { modification_time: ts as i64, data_change: true, stats: None, - stats_parsed: None, tags: None, deletion_vector: None, base_row_id: None, @@ -504,7 +501,7 @@ async fn validate_lock_table_state(table: &DeltaTable, expected_version: i64) -> let latest = client .get_latest_entries(&table.table_uri(), WORKERS * COMMITS) .await?; - let max_version = latest.get(0).unwrap().version; + let max_version = latest.first().unwrap().version; assert_eq!(max_version, expected_version); // Pull out pairs of consecutive commit entries and verify invariants. diff --git a/crates/aws/tests/repair_s3_rename_test.rs b/crates/aws/tests/repair_s3_rename_test.rs index 0b141c775b..136a56e50e 100644 --- a/crates/aws/tests/repair_s3_rename_test.rs +++ b/crates/aws/tests/repair_s3_rename_test.rs @@ -2,9 +2,9 @@ use bytes::Bytes; use deltalake_aws::storage::S3StorageBackend; -use deltalake_core::storage::object_store::{ - DynObjectStore, Error as ObjectStoreError, GetOptions, GetResult, ListResult, MultipartId, - ObjectMeta, PutOptions, PutResult, Result as ObjectStoreResult, +use deltalake_core::logstore::object_store::{ + DynObjectStore, Error as ObjectStoreError, GetOptions, GetResult, ListResult, ObjectMeta, + PutOptions, PutResult, Result as ObjectStoreResult, }; use deltalake_core::{DeltaTableBuilder, ObjectStore, Path}; use deltalake_test::utils::IntegrationContext; @@ -13,7 +13,6 @@ use object_store::{MultipartUpload, PutMultipartOpts, PutPayload}; use serial_test::serial; use std::ops::Range; use std::sync::{Arc, Mutex}; -use tokio::io::AsyncWrite; use tokio::task::JoinHandle; use tokio::time::Duration; @@ -188,7 +187,7 @@ impl ObjectStore for DelayedObjectStore { self.inner.get_opts(location, options).await } - async fn get_range(&self, location: &Path, range: Range) -> ObjectStoreResult { + async fn get_range(&self, location: &Path, range: Range) -> ObjectStoreResult { self.inner.get_range(location, range).await } @@ -200,7 +199,7 @@ impl ObjectStore for DelayedObjectStore { self.inner.delete(location).await } - fn list(&self, prefix: Option<&Path>) -> BoxStream<'_, ObjectStoreResult> { + fn list(&self, prefix: Option<&Path>) -> BoxStream<'static, ObjectStoreResult> { self.inner.list(prefix) } @@ -208,7 +207,7 @@ impl ObjectStore for DelayedObjectStore { &self, prefix: Option<&Path>, offset: &Path, - ) -> BoxStream<'_, ObjectStoreResult> { + ) -> BoxStream<'static, ObjectStoreResult> { self.inner.list_with_offset(prefix, offset) } diff --git a/crates/azure/Cargo.toml b/crates/azure/Cargo.toml index d7e6992353..43aa953c2a 100644 --- a/crates/azure/Cargo.toml +++ b/crates/azure/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "deltalake-azure" -version = "0.8.0" +version = "0.9.1" authors.workspace = true keywords.workspace = true readme.workspace = true @@ -12,9 +12,7 @@ repository.workspace = true rust-version.workspace = true [dependencies] -deltalake-core = { version = "0.25.0", path = "../core", features = [ - "datafusion", "cloud", -]} +deltalake-core = { version = "0.26.0", path = "../core" } # workspace depenndecies async-trait = { workspace = true } @@ -28,6 +26,9 @@ regex = { workspace = true } url = { workspace = true } [dev-dependencies] +deltalake-core = { version = "0.26.0", path = "../core", features = [ + "datafusion", +] } chrono = { workspace = true } serial_test = "3" deltalake-test = { path = "../test" } diff --git a/crates/azure/src/lib.rs b/crates/azure/src/lib.rs index 1d8744635f..2d0bc51dbf 100644 --- a/crates/azure/src/lib.rs +++ b/crates/azure/src/lib.rs @@ -2,14 +2,15 @@ use std::collections::HashMap; use std::str::FromStr; use std::sync::Arc; -use deltalake_core::logstore::{default_logstore, logstores, LogStore, LogStoreFactory}; -use deltalake_core::storage::{ - factories, limit_store_handler, url_prefix_handler, ObjectStoreFactory, ObjectStoreRef, - RetryConfigParse, StorageOptions, +use deltalake_core::logstore::{ + default_logstore, logstore_factories, object_store_factories, LogStore, LogStoreFactory, + ObjectStoreFactory, ObjectStoreRef, StorageConfig, }; use deltalake_core::{DeltaResult, DeltaTableError, Path}; use object_store::azure::{AzureConfigKey, MicrosoftAzureBuilder}; -use object_store::ObjectStoreScheme; +use object_store::client::SpawnedReqwestConnector; +use object_store::{ObjectStoreScheme, RetryConfig}; +use tokio::runtime::Handle; use url::Url; mod config; @@ -19,10 +20,9 @@ trait AzureOptions { fn as_azure_options(&self) -> HashMap; } -impl AzureOptions for StorageOptions { +impl AzureOptions for HashMap { fn as_azure_options(&self) -> HashMap { - self.0 - .iter() + self.iter() .filter_map(|(key, value)| { Some(( AzureConfigKey::from_str(&key.to_ascii_lowercase()).ok()?, @@ -36,45 +36,53 @@ impl AzureOptions for StorageOptions { #[derive(Clone, Default, Debug)] pub struct AzureFactory {} -impl RetryConfigParse for AzureFactory {} - impl ObjectStoreFactory for AzureFactory { fn parse_url_opts( &self, url: &Url, - options: &StorageOptions, + options: &HashMap, + retry: &RetryConfig, + handle: Option, ) -> DeltaResult<(ObjectStoreRef, Path)> { let config = config::AzureConfigHelper::try_new(options.as_azure_options())?.build()?; - let (_, path) = - ObjectStoreScheme::parse(url).map_err(|e| DeltaTableError::GenericError { - source: Box::new(e), - })?; - let prefix = Path::parse(path)?; + let mut builder = MicrosoftAzureBuilder::new() + .with_url(url.to_string()) + .with_retry(retry.clone()); - let mut builder = MicrosoftAzureBuilder::new().with_url(url.to_string()); + if let Some(handle) = handle { + builder = builder.with_http_connector(SpawnedReqwestConnector::new(handle)); + } for (key, value) in config.iter() { builder = builder.with_config(*key, value.clone()); } + let store = builder.build()?; - let inner = builder - .with_retry(self.parse_retry_config(options)?) - .build()?; + let (_, path) = + ObjectStoreScheme::parse(url).map_err(|e| DeltaTableError::GenericError { + source: Box::new(e), + })?; + let prefix = Path::parse(path)?; - let store = limit_store_handler(url_prefix_handler(inner, prefix.clone()), options); - Ok((store, prefix)) + Ok((Arc::new(store), prefix)) } } impl LogStoreFactory for AzureFactory { fn with_options( &self, - store: ObjectStoreRef, + prefixed_store: ObjectStoreRef, + root_store: ObjectStoreRef, location: &Url, - options: &StorageOptions, + options: &StorageConfig, ) -> DeltaResult> { - Ok(default_logstore(store, location, options)) + Ok(default_logstore( + prefixed_store, + root_store, + location, + options, + )) } } @@ -83,7 +91,7 @@ pub fn register_handlers(_additional_prefixes: Option) { let factory = Arc::new(AzureFactory {}); for scheme in ["az", "adl", "azure", "abfs", "abfss"].iter() { let url = Url::parse(&format!("{scheme}://")).unwrap(); - factories().insert(url.clone(), factory.clone()); - logstores().insert(url.clone(), factory.clone()); + object_store_factories().insert(url.clone(), factory.clone()); + logstore_factories().insert(url.clone(), factory.clone()); } } diff --git a/crates/azure/tests/integration.rs b/crates/azure/tests/integration.rs index b07a2faf1a..6a972c354f 100644 --- a/crates/azure/tests/integration.rs +++ b/crates/azure/tests/integration.rs @@ -46,7 +46,6 @@ async fn test_concurrency_azure() -> TestResult { // comment](https://github.com/delta-io/delta-rs/pull/1564#issuecomment-1721048753) and we should // figure out a way to re-enable this test at least in the GitHub Actions CI environment #[ignore] -#[cfg(feature = "azure")] #[tokio::test] #[serial] async fn test_object_store_onelake() -> TestResult { @@ -60,7 +59,6 @@ async fn test_object_store_onelake() -> TestResult { // comment](https://github.com/delta-io/delta-rs/pull/1564#issuecomment-1721048753) and we should // figure out a way to re-enable this test at least in the GitHub Actions CI environment #[ignore] -#[cfg(feature = "azure")] #[tokio::test] #[serial] async fn test_object_store_onelake_abfs() -> TestResult { @@ -72,7 +70,7 @@ async fn test_object_store_onelake_abfs() -> TestResult { #[allow(dead_code)] async fn read_write_test_onelake(context: &IntegrationContext, path: &Path) -> TestResult { - let delta_store = DeltaTableBuilder::from_uri(&context.root_uri()) + let delta_store = DeltaTableBuilder::from_uri(context.root_uri()) .with_allow_http(true) .build_storage()? .object_store(None); @@ -87,7 +85,8 @@ async fn read_write_test_onelake(context: &IntegrationContext, path: &Path) -> T assert_eq!(expected, fetched); for range in [0..10, 3..5, 0..expected.len()] { - let data = delta_store.get_range(path, range.clone()).await.unwrap(); + let range_u64 = range.start as u64..range.end as u64; + let data = delta_store.get_range(path, range_u64).await.unwrap(); assert_eq!(&data[..], &expected[range]) } @@ -106,7 +105,7 @@ fn list_delta_tables_using_listing_provider_with_missing_account_name() -> TestR let storage_options = HashMap::::new(); if let Err(read_error) = - ListingSchemaProvider::try_new(&context.root_uri(), Some(storage_options)) + ListingSchemaProvider::try_new(context.root_uri(), Some(storage_options)) { assert_eq!(read_error.to_string(), "Failed to read delta log object: Generic MicrosoftAzure error: Account must be specified".to_string()); }; @@ -125,7 +124,7 @@ async fn list_delta_tables_using_listing_provider_with_account_name() -> TestRes let mut storage_options = HashMap::::new(); storage_options.insert("account_name".to_string(), "test_account".to_string()); - let schema = ListingSchemaProvider::try_new(&context.root_uri(), Some(storage_options)); + let schema = ListingSchemaProvider::try_new(context.root_uri(), Some(storage_options)); assert!( schema.is_ok(), "Capable of reading the storage options. Fails if e.g. `account_name` is missing" diff --git a/crates/benchmarks/README.md b/crates/benchmarks/README.md index c5d6b0b920..959a857977 100644 --- a/crates/benchmarks/README.md +++ b/crates/benchmarks/README.md @@ -14,7 +14,7 @@ You can generate the TPC-DS dataset yourself by downloading and compiling [the g You may need to update the CFLAGS to include `-fcommon` to compile on newer versions of GCC. ## Commands -These commands can be executed from the root of the benchmark crate. Some commands depend on the existance of the TPC-DS Dataset existing. +These commands can be executed from the root of the benchmark crate. Some commands depend on the existence of the TPC-DS Dataset existing. ### Convert Converts a TPC-DS web_returns csv into a Delta table @@ -33,7 +33,7 @@ group_id: Used to group all tests that executed as a part of this call. Default name: The benchmark name that was executed sample: The iteration number for a given benchmark name duration_ms: How long the benchmark took in ms -data: Free field to pack any additonal data +data: Free field to pack any additional data ``` cargo run --release --bin merge -- standard data/web_returns 1 data/merge_results diff --git a/crates/catalog-glue/Cargo.toml b/crates/catalog-glue/Cargo.toml index fef12a8c48..9673b1c5cd 100644 --- a/crates/catalog-glue/Cargo.toml +++ b/crates/catalog-glue/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "deltalake-catalog-glue" -version = "0.9.0" +version = "0.10.0" authors.workspace = true keywords.workspace = true readme.workspace = true @@ -15,7 +15,7 @@ rust-version.workspace = true async-trait = { workspace = true } aws-config = "1" aws-sdk-glue = "1" -deltalake-core = { version = "0.25.0", path = "../core" } +deltalake-core = { version = "0.26.0", path = "../core" } thiserror = { workspace = true } [dev-dependencies] diff --git a/crates/catalog-unity/Cargo.toml b/crates/catalog-unity/Cargo.toml index 444d0ccfb6..31933bb873 100644 --- a/crates/catalog-unity/Cargo.toml +++ b/crates/catalog-unity/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "deltalake-catalog-unity" -version = "0.9.0" +version = "0.10.1" authors.workspace = true keywords.workspace = true readme.workspace = true @@ -20,12 +20,10 @@ thiserror.workspace = true futures.workspace = true chrono.workspace = true tracing.workspace = true -deltalake-core = { version = "0.25.0", path = "../core", features = [ - "datafusion", -] } -deltalake-aws = { version = "0.8.0", path = "../aws", optional = true } -deltalake-azure = { version = "0.8.0", path = "../azure", optional = true } -deltalake-gcp = { version = "0.9.0", path = "../gcp", optional = true } +deltalake-core = { version = "0.26.0", path = "../core" } +deltalake-aws = { version = "0.9.0", path = "../aws", optional = true } +deltalake-azure = { version = "0.9.0", path = "../azure", optional = true } +deltalake-gcp = { version = "0.10.0", path = "../gcp", optional = true } reqwest = { version = "0.12", default-features = false, features = ["rustls-tls", "json", "http2"] } reqwest-retry = "0.7" reqwest-middleware = { version = "0.4.0", features = ["json"] } diff --git a/crates/catalog-unity/src/client/retry.rs b/crates/catalog-unity/src/client/retry.rs index 9b3828274e..aed26db9d0 100644 --- a/crates/catalog-unity/src/client/retry.rs +++ b/crates/catalog-unity/src/client/retry.rs @@ -52,7 +52,7 @@ impl From for std::io::Error { (_, Some(StatusCode::BAD_REQUEST)) => Self::new(ErrorKind::InvalidInput, err), (Some(source), None) if source.is_timeout() => Self::new(ErrorKind::TimedOut, err), (Some(source), None) if source.is_connect() => Self::new(ErrorKind::NotConnected, err), - _ => Self::new(ErrorKind::Other, err), + _ => Self::other(err), } } } diff --git a/crates/catalog-unity/src/datafusion.rs b/crates/catalog-unity/src/datafusion.rs index 41486b979b..0b07bded5d 100644 --- a/crates/catalog-unity/src/datafusion.rs +++ b/crates/catalog-unity/src/datafusion.rs @@ -162,7 +162,7 @@ impl UnitySchemaProvider { { ListTableSummariesResponse::Success { tables, .. } => tables .into_iter() - .filter_map(|t| t.full_name.split('.').last().map(|n| n.into())) + .filter_map(|t| t.full_name.split('.').next_back().map(|n| n.into())) .collect(), ListTableSummariesResponse::Error(_) => vec![], }; diff --git a/crates/catalog-unity/src/lib.rs b/crates/catalog-unity/src/lib.rs index 5d76c98a8b..8f589f952d 100644 --- a/crates/catalog-unity/src/lib.rs +++ b/crates/catalog-unity/src/lib.rs @@ -7,14 +7,17 @@ compile_error!( for this crate to function properly." ); -use datafusion_common::DataFusionError; -use deltalake_core::logstore::{default_logstore, logstores, LogStore, LogStoreFactory}; +use deltalake_core::logstore::{ + default_logstore, logstore_factories, object_store::RetryConfig, LogStore, LogStoreFactory, + StorageConfig, +}; use reqwest::header::{HeaderValue, InvalidHeaderValue, AUTHORIZATION}; use reqwest::Url; use std::collections::HashMap; use std::future::Future; use std::str::FromStr; use std::sync::Arc; +use tokio::runtime::Handle; use crate::credential::{ AzureCliCredential, ClientSecretOAuthProvider, CredentialProvider, WorkspaceOAuthProvider, @@ -31,9 +34,8 @@ use deltalake_core::{ }; use crate::client::retry::*; -use deltalake_core::storage::{ - factories, str_is_truthy, IORuntime, ObjectStoreFactory, ObjectStoreRef, RetryConfigParse, - StorageOptions, +use deltalake_core::logstore::{ + config::str_is_truthy, object_store_factories, IORuntime, ObjectStoreFactory, ObjectStoreRef, }; pub mod client; pub mod credential; @@ -554,11 +556,9 @@ impl UnityCatalogBuilder { .get_temp_table_credentials(catalog_id, database_name, table_name) .await?; let credentials = match temp_creds_res { - TableTempCredentialsResponse::Success(temp_creds) => { - temp_creds.get_credentials().ok_or_else(|| { - DataFusionError::External(UnityCatalogError::MissingCredential.into()) - })? - } + TableTempCredentialsResponse::Success(temp_creds) => temp_creds + .get_credentials() + .ok_or_else(|| UnityCatalogError::MissingCredential)?, TableTempCredentialsResponse::Error(_error) => { return Err(UnityCatalogError::TemporaryCredentialsFetchFailure) } @@ -835,30 +835,33 @@ impl UnityCatalog { #[derive(Clone, Default, Debug)] pub struct UnityCatalogFactory {} -impl RetryConfigParse for UnityCatalogFactory {} - impl ObjectStoreFactory for UnityCatalogFactory { fn parse_url_opts( &self, table_uri: &Url, - options: &StorageOptions, + options: &HashMap, + _retry: &RetryConfig, + handle: Option, ) -> DeltaResult<(ObjectStoreRef, Path)> { let (table_path, temp_creds) = UnityCatalogBuilder::execute_uc_future( UnityCatalogBuilder::get_uc_location_and_token(table_uri.as_str()), - )? - .map_err(UnityCatalogError::from)?; + )??; - let mut storage_options = options.0.clone(); + let mut storage_options = options.clone(); storage_options.extend(temp_creds); - let mut builder = - DeltaTableBuilder::from_uri(&table_path).with_io_runtime(IORuntime::default()); + // TODO(roeap): we should not have to go through the table here. + // ideally we just create the right storage ... + let mut builder = DeltaTableBuilder::from_uri(&table_path); + + if let Some(handle) = handle { + builder = builder.with_io_runtime(IORuntime::RT(handle)); + } if !storage_options.is_empty() { builder = builder.with_storage_options(storage_options.clone()); } - let prefix = Path::parse(table_uri.path())?; - let store = builder.build()?.object_store(); + let store = builder.build_storage()?.object_store(None); Ok((store, prefix)) } @@ -867,21 +870,26 @@ impl ObjectStoreFactory for UnityCatalogFactory { impl LogStoreFactory for UnityCatalogFactory { fn with_options( &self, - store: ObjectStoreRef, + prefixed_store: ObjectStoreRef, + root_store: ObjectStoreRef, location: &Url, - options: &StorageOptions, + options: &StorageConfig, ) -> DeltaResult> { - Ok(default_logstore(store, location, options)) + Ok(default_logstore( + prefixed_store, + root_store, + location, + options, + )) } } /// Register an [ObjectStoreFactory] for common UnityCatalogFactory [Url] schemes pub fn register_handlers(_additional_prefixes: Option) { let factory = Arc::new(UnityCatalogFactory::default()); - let scheme = "uc"; - let url = Url::parse(&format!("{scheme}://")).unwrap(); - factories().insert(url.clone(), factory.clone()); - logstores().insert(url.clone(), factory.clone()); + let url = Url::parse("uc://").unwrap(); + object_store_factories().insert(url.clone(), factory.clone()); + logstore_factories().insert(url.clone(), factory.clone()); } #[async_trait::async_trait] diff --git a/crates/core/Cargo.toml b/crates/core/Cargo.toml index d1ccf79f2e..4a544f4a15 100644 --- a/crates/core/Cargo.toml +++ b/crates/core/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "deltalake-core" -version = "0.25.0" +version = "0.26.2" authors.workspace = true keywords.workspace = true readme.workspace = true @@ -12,9 +12,11 @@ repository.workspace = true rust-version.workspace = true [package.metadata.docs.rs] -features = ["datafusion", "json", "unity-experimental"] +features = ["datafusion", "json"] [dependencies] +deltalake-derive = { version = "0.26.0", path = "../derive" } + delta_kernel.workspace = true # arrow @@ -29,10 +31,8 @@ arrow-ord = { workspace = true } arrow-row = { workspace = true } arrow-schema = { workspace = true, features = ["serde"] } arrow-select = { workspace = true } -parquet = { workspace = true, features = [ - "async", - "object_store", -] } +parquet = { workspace = true, features = ["async", "object_store"] } +object_store = { workspace = true } pin-project-lite = "^0.2.7" # datafusion @@ -49,7 +49,7 @@ datafusion-functions-aggregate = { workspace = true, optional = true } # serde serde = { workspace = true, features = ["derive"] } serde_json = { workspace = true } -strum = { workspace = true} +strum = { workspace = true } # "stdlib" bytes = { workspace = true } @@ -57,7 +57,7 @@ chrono = { workspace = true, default-features = false, features = ["clock"] } regex = { workspace = true } thiserror = { workspace = true } uuid = { workspace = true, features = ["serde", "v4"] } -url = { workspace = true } +url = { workspace = true, features = ["serde"] } urlencoding = { workspace = true } # runtime @@ -75,45 +75,44 @@ tokio = { workspace = true, features = [ "parking_lot", ] } +# caching +foyer = { version = "0.17.2", optional = true, features = ["serde"] } +tempfile = { version = "3.19.1", optional = true } + # other deps (these should be organized and pulled into workspace.dependencies as necessary) cfg-if = "1" dashmap = "6" -errno = "0.3" either = "1.8" -fix-hidden-lifetime-bug = "0.2" indexmap = "2.2.1" itertools = "0.14" -libc = ">=0.2.90, <1" num-bigint = "0.4" num-traits = "0.2.15" -object_store = { workspace = true } parking_lot = "0.12" percent-encoding = "2" -roaring = "0.10.1" tracing = { workspace = true } rand = "0.8" -z85 = "3.0.5" maplit = "1" -sqlparser = { version = "0.53.0" } +sqlparser = { version = "0.56.0" } humantime = { version = "2.1.0" } +validator = { version = "0.19", features = ["derive"] } [dev-dependencies] criterion = "0.5" ctor = "0" -deltalake-test = { path = "../test", features = ["datafusion"] } +datatest-stable = "0.2" +deltalake-test = { path = "../test" } dotenvy = "0" fs_extra = "1.2.0" maplit = "1" pretty_assertions = "1.2.1" pretty_env_logger = "0.5.0" -rand = "0.8" +rstest = { version = "0.25.0" } serial_test = "3" tempfile = "3" tokio = { version = "1", features = ["macros", "rt-multi-thread"] } [features] -cdf = [] -default = ["cdf", "rustls"] +default = ["rustls"] datafusion = [ "dep:datafusion", "datafusion-expr", @@ -131,3 +130,56 @@ python = ["arrow/pyarrow"] native-tls = ["delta_kernel/default-engine"] rustls = ["delta_kernel/default-engine-rustls"] cloud = ["object_store/cloud"] + +# enable caching some file I/O operations when scanning delta logs +delta-cache = ["foyer", "tempfile", "url/serde"] + +integration_test = [] + +[[test]] +name = "dat" +harness = false + +[[test]] +name = "command_optimize" +required-features = ["datafusion"] + +[[test]] +name = "command_merge" +required-features = ["datafusion"] + +[[test]] +name = "command_restore" +required-features = ["datafusion"] + +[[test]] +name = "checkpoint_writer" +required-features = ["datafusion"] + +[[test]] +name = "command_filesystem_check" +required-features = ["datafusion"] + +[[test]] +name = "command_vacuum" +required-features = ["datafusion"] + +[[test]] +name = "commit_info_format" +required-features = ["datafusion"] + +[[test]] +name = "integration" +required-features = ["datafusion"] + +[[test]] +name = "integration_checkpoint" +required-features = ["datafusion"] + +[[test]] +name = "read_delta_log_test" +required-features = ["datafusion"] + +[[test]] +name = "read_delta_partitions_test" +required-features = ["datafusion"] diff --git a/crates/core/src/data_catalog/storage/mod.rs b/crates/core/src/data_catalog/storage/mod.rs index 3e08c5f463..996331aa46 100644 --- a/crates/core/src/data_catalog/storage/mod.rs +++ b/crates/core/src/data_catalog/storage/mod.rs @@ -13,8 +13,8 @@ use futures::TryStreamExt; use object_store::ObjectStore; use crate::errors::DeltaResult; +use crate::logstore::{store_for, StorageConfig}; use crate::open_table_with_storage_options; -use crate::storage::*; use crate::table::builder::ensure_table_uri; const DELTA_LOG_FOLDER: &str = "_delta_log"; @@ -38,24 +38,23 @@ pub struct ListingSchemaProvider { /// A map of table names to a fully quilfied storage location tables: DashMap, /// Options used to create underlying object stores - storage_options: StorageOptions, + storage_options: StorageConfig, } impl ListingSchemaProvider { /// Create a new [`ListingSchemaProvider`] pub fn try_new( root_uri: impl AsRef, - storage_options: Option>, + options: Option>, ) -> DeltaResult { let uri = ensure_table_uri(root_uri)?; - let storage_options: StorageOptions = storage_options.unwrap_or_default().into(); - // We already parsed the url, so unwrapping is safe. - let store = store_for(&uri, &storage_options)?; + let options = options.unwrap_or_default(); + let store = store_for(&uri, &options)?; Ok(Self { authority: uri.to_string(), store, tables: DashMap::new(), - storage_options, + storage_options: StorageConfig::parse_options(options)?, }) } @@ -116,7 +115,7 @@ impl SchemaProvider for ListingSchemaProvider { return Ok(None); }; let provider = - open_table_with_storage_options(location, self.storage_options.0.clone()).await?; + open_table_with_storage_options(location, self.storage_options.raw.clone()).await?; Ok(Some(Arc::new(provider) as Arc)) } diff --git a/crates/core/src/delta_datafusion/cdf/mod.rs b/crates/core/src/delta_datafusion/cdf/mod.rs index aeef5918d8..5a973c1ca4 100644 --- a/crates/core/src/delta_datafusion/cdf/mod.rs +++ b/crates/core/src/delta_datafusion/cdf/mod.rs @@ -103,11 +103,9 @@ impl FileAction for Remove { } else { match self.partition_values { Some(ref part_map) => Ok(part_map), - _ => Err(crate::DeltaTableError::Protocol { - source: crate::protocol::ProtocolError::InvalidField( - "partition_values".to_string(), - ), - }), + _ => Err(crate::DeltaTableError::MetadataError( + "Remove action is missing required field: 'partition_values'".to_string(), + )), } } } @@ -123,9 +121,9 @@ impl FileAction for Remove { } else { match self.size { Some(size) => Ok(size as usize), - _ => Err(crate::DeltaTableError::Protocol { - source: crate::protocol::ProtocolError::InvalidField("size".to_string()), - }), + _ => Err(crate::DeltaTableError::MetadataError( + "Remove action is missing required field: 'size'".to_string(), + )), } } } diff --git a/crates/core/src/delta_datafusion/cdf/scan_utils.rs b/crates/core/src/delta_datafusion/cdf/scan_utils.rs index 459157f4c8..91a6fbf5f9 100644 --- a/crates/core/src/delta_datafusion/cdf/scan_utils.rs +++ b/crates/core/src/delta_datafusion/cdf/scan_utils.rs @@ -75,7 +75,7 @@ pub fn create_partition_values( let part = PartitionedFile { object_meta: ObjectMeta { location: Path::parse(action.path().as_str())?, - size: action.size()?, + size: action.size()? as u64, e_tag: None, last_modified: chrono::Utc.timestamp_nanos(0), version: None, diff --git a/crates/core/src/delta_datafusion/expr.rs b/crates/core/src/delta_datafusion/expr.rs index 9819e793e7..84006c5b2b 100644 --- a/crates/core/src/delta_datafusion/expr.rs +++ b/crates/core/src/delta_datafusion/expr.rs @@ -333,7 +333,7 @@ impl Display for BinaryExprFormat<'_> { impl Display for SqlFormat<'_> { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self.expr { - Expr::Column(c) => write!(f, "{c}"), + Expr::Column(c) => write!(f, "{}", c.quoted_flat_name()), Expr::Literal(v) => write!(f, "{}", ScalarValueFormat { scalar: v }), Expr::Case(case) => { write!(f, "CASE ")?; @@ -638,7 +638,7 @@ mod test { ), StructField::new( "money".to_string(), - DataType::Primitive(PrimitiveType::Decimal(12, 2)), + DataType::Primitive(PrimitiveType::decimal(12, 2).unwrap()), true, ), StructField::new( @@ -663,7 +663,7 @@ mod test { ), StructField::new( "_decimal".to_string(), - DataType::Primitive(PrimitiveType::Decimal(2, 2)), + DataType::Primitive(PrimitiveType::decimal(2, 2).unwrap()), true, ), StructField::new( @@ -697,7 +697,7 @@ mod test { .with_columns(schema.fields().cloned()) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); table } @@ -727,7 +727,7 @@ mod test { }, simple!( Expr::Column(Column::from_qualified_name_ignore_case("Value3")).eq(lit(3_i64)), - "Value3 = 3".to_string() + "\"Value3\" = 3".to_string() ), simple!(col("active").is_true(), "active IS TRUE".to_string()), simple!(col("active"), "active".to_string()), diff --git a/crates/core/src/delta_datafusion/logical.rs b/crates/core/src/delta_datafusion/logical.rs index 1f22e8e75d..fe181bfad0 100644 --- a/crates/core/src/delta_datafusion/logical.rs +++ b/crates/core/src/delta_datafusion/logical.rs @@ -50,15 +50,6 @@ impl UserDefinedLogicalNodeCore for MetricObserver { write!(f, "MetricObserver id={}", self.id) } - fn from_template( - &self, - exprs: &[datafusion_expr::Expr], - inputs: &[datafusion_expr::LogicalPlan], - ) -> Self { - self.with_exprs_and_inputs(exprs.to_vec(), inputs.to_vec()) - .unwrap() - } - fn with_exprs_and_inputs( &self, _exprs: Vec, diff --git a/crates/core/src/delta_datafusion/mod.rs b/crates/core/src/delta_datafusion/mod.rs index a3286a2b78..0d67befb71 100644 --- a/crates/core/src/delta_datafusion/mod.rs +++ b/crates/core/src/delta_datafusion/mod.rs @@ -27,7 +27,9 @@ use std::fmt::{self, Debug}; use std::sync::Arc; use arrow_array::types::UInt16Type; -use arrow_array::{Array, DictionaryArray, RecordBatch, StringArray, TypedDictionaryArray}; +use arrow_array::{ + Array, BooleanArray, DictionaryArray, RecordBatch, StringArray, TypedDictionaryArray, +}; use arrow_cast::display::array_value_to_string; use arrow_cast::{cast_with_options, CastOptions}; use arrow_schema::{ @@ -35,19 +37,22 @@ use arrow_schema::{ SchemaRef as ArrowSchemaRef, TimeUnit, }; use arrow_select::concat::concat_batches; +use arrow_select::filter::filter_record_batch; use async_trait::async_trait; use chrono::{DateTime, TimeZone, Utc}; +use datafusion::catalog::memory::DataSourceExec; use datafusion::catalog::{Session, TableProviderFactory}; use datafusion::config::TableParquetOptions; use datafusion::datasource::physical_plan::{ - wrap_partition_type_in_dict, wrap_partition_value_in_dict, FileScanConfig, ParquetSource, + wrap_partition_type_in_dict, wrap_partition_value_in_dict, FileGroup, FileScanConfigBuilder, + ParquetSource, }; use datafusion::datasource::{listing::PartitionedFile, MemTable, TableProvider, TableType}; use datafusion::execution::context::{SessionConfig, SessionContext, SessionState, TaskContext}; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; use datafusion::optimizer::simplify_expressions::ExprSimplifier; -use datafusion::physical_optimizer::pruning::PruningPredicate; +use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; use datafusion_common::scalar::ScalarValue; use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion, TreeNodeVisitor}; use datafusion_common::{ @@ -57,14 +62,15 @@ use datafusion_common::{ use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::logical_plan::CreateExternalTable; use datafusion_expr::simplify::SimplifyContext; -use datafusion_expr::utils::conjunction; -use datafusion_expr::{col, Expr, Extension, LogicalPlan, TableProviderFilterPushDown, Volatility}; -use datafusion_physical_expr::{create_physical_expr, PhysicalExpr}; +use datafusion_expr::utils::{conjunction, split_conjunction}; +use datafusion_expr::{ + col, BinaryExpr, Expr, Extension, LogicalPlan, Operator, TableProviderFilterPushDown, + Volatility, +}; +use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_plan::filter::FilterExec; -use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; -use datafusion_physical_plan::memory::{LazyBatchGenerator, LazyMemoryExec}; +use datafusion_physical_plan::limit::LocalLimitExec; use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; -use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, SendableRecordBatchStream, Statistics, @@ -72,11 +78,11 @@ use datafusion_physical_plan::{ use datafusion_proto::logical_plan::LogicalExtensionCodec; use datafusion_proto::physical_plan::PhysicalExtensionCodec; use datafusion_sql::planner::ParserOptions; +use delta_kernel::engine::arrow_conversion::TryIntoArrow as _; use either::Either; use futures::TryStreamExt; use itertools::Itertools; use object_store::ObjectMeta; -use parking_lot::RwLock; use serde::{Deserialize, Serialize}; use url::Url; @@ -84,7 +90,9 @@ use url::Url; use crate::delta_datafusion::expr::parse_predicate_expression; use crate::delta_datafusion::schema_adapter::DeltaSchemaAdapterFactory; use crate::errors::{DeltaResult, DeltaTableError}; -use crate::kernel::{Add, DataCheck, EagerSnapshot, Invariant, Snapshot, StructTypeExt}; +use crate::kernel::{ + Add, DataCheck, EagerSnapshot, Invariant, LogDataHandler, Snapshot, StructTypeExt, +}; use crate::logstore::LogStoreRef; use crate::table::builder::ensure_table_uri; use crate::table::state::DeltaTableState; @@ -205,13 +213,13 @@ fn _arrow_schema(snapshot: &Snapshot, wrap_partitions: bool) -> DeltaResult DeltaScanBuilder<'a> { for idx in used_columns { fields.push(logical_schema.field(*idx).to_owned()); } + // partition filters with Exact pushdown were removed from projection by DF optimizer, + // we need to add them back for the predicate pruning to work + if let Some(expr) = &self.filter { + for c in expr.column_refs() { + let idx = logical_schema.index_of(c.name.as_str())?; + if !used_columns.contains(&idx) { + fields.push(logical_schema.field(idx).to_owned()); + } + } + } Arc::new(ArrowSchema::new(fields)) } else { logical_schema @@ -547,52 +565,101 @@ impl<'a> DeltaScanBuilder<'a> { let context = SessionContext::new(); let df_schema = logical_schema.clone().to_dfschema()?; - let logical_filter = self.filter.map(|expr| { - // Simplify the expression first - let props = ExecutionProps::new(); - let simplify_context = - SimplifyContext::new(&props).with_schema(df_schema.clone().into()); - let simplifier = ExprSimplifier::new(simplify_context).with_max_cycles(10); - let simplified = simplifier.simplify(expr).unwrap(); + let logical_filter = self + .filter + .clone() + .map(|expr| simplify_expr(&context, &df_schema, expr)); + // only inexact filters should be pushed down to the data source, doing otherwise + // will make stats inexact and disable datafusion optimizations like AggregateStatistics + let pushdown_filter = self + .filter + .and_then(|expr| { + let predicates = split_conjunction(&expr); + let pushdown_filters = get_pushdown_filters( + &predicates, + self.snapshot.metadata().partition_columns.as_slice(), + ); - context - .create_physical_expr(simplified, &df_schema) - .unwrap() - }); + let filtered_predicates = predicates + .into_iter() + .zip(pushdown_filters.into_iter()) + .filter_map(|(filter, pushdown)| { + if pushdown == TableProviderFilterPushDown::Inexact { + Some(filter.clone()) + } else { + None + } + }); + conjunction(filtered_predicates) + }) + .map(|expr| simplify_expr(&context, &df_schema, expr)); // Perform Pruning of files to scan - let (files, files_scanned, files_pruned) = match self.files { + let (files, files_scanned, files_pruned, pruning_mask) = match self.files { Some(files) => { let files = files.to_owned(); let files_scanned = files.len(); - (files, files_scanned, 0) + (files, files_scanned, 0, None) } None => { - if let Some(predicate) = &logical_filter { - let pruning_predicate = - PruningPredicate::try_new(predicate.clone(), logical_schema.clone())?; - let files_to_prune = pruning_predicate.prune(self.snapshot)?; - let mut files_pruned = 0usize; - let files = self + // early return in case we have no push down filters or limit + if logical_filter.is_none() && self.limit.is_none() { + let files = self.snapshot.file_actions()?; + let files_scanned = files.len(); + (files, files_scanned, 0, None) + } else { + let num_containers = self.snapshot.num_containers(); + + let files_to_prune = if let Some(predicate) = &logical_filter { + let pruning_predicate = + PruningPredicate::try_new(predicate.clone(), logical_schema.clone())?; + pruning_predicate.prune(self.snapshot)? + } else { + vec![true; num_containers] + }; + + // needed to enforce limit and deal with missing statistics + // rust port of https://github.com/delta-io/delta/pull/1495 + let mut pruned_without_stats = vec![]; + let mut rows_collected = 0; + let mut files = vec![]; + + for (action, keep) in self .snapshot .file_actions_iter()? - .zip(files_to_prune.into_iter()) - .filter_map(|(action, keep)| { - if keep { - Some(action.to_owned()) + .zip(files_to_prune.iter().cloned()) + { + // prune file based on predicate pushdown + if keep { + // prune file based on limit pushdown + if let Some(limit) = self.limit { + if let Some(stats) = action.get_stats()? { + if rows_collected <= limit as i64 { + rows_collected += stats.num_records; + files.push(action.to_owned()); + } else { + break; + } + } else { + // some files are missing stats; skipping but storing them + // in a list in case we can't reach the target limit + pruned_without_stats.push(action.to_owned()); + } } else { - files_pruned += 1; - None + files.push(action.to_owned()); } - }) - .collect::>(); + } + } + + if let Some(limit) = self.limit { + if rows_collected < limit as i64 { + files.extend(pruned_without_stats); + } + } let files_scanned = files.len(); - (files, files_scanned, files_pruned) - } else { - let files = self.snapshot.file_actions()?; - let files_scanned = files.len(); - (files, files_scanned, 0) + let files_pruned = num_containers - files_scanned; + (files, files_scanned, files_pruned, Some(files_to_prune)) } } }; @@ -649,10 +716,18 @@ impl<'a> DeltaScanBuilder<'a> { )); } - let stats = self - .snapshot - .datafusion_table_statistics() - .unwrap_or(Statistics::new_unknown(&schema)); + // FIXME - where is the correct place to marry file pruning with statistics pruning? + // Temporarily re-generating the log handler, just so that we can compute the stats. + // Should we update datafusion_table_statistics to optionally take the mask? + let stats = if let Some(mask) = pruning_mask { + let es = self.snapshot.snapshot(); + let pruned_stats = prune_file_statistics(&es.files, mask); + LogDataHandler::new(&pruned_stats, es.metadata(), es.schema()).statistics() + } else { + self.snapshot.datafusion_table_statistics() + }; + + let stats = stats.unwrap_or(Statistics::new_unknown(&schema)); let parquet_options = TableParquetOptions { global: self.session.config().options().execution.parquet.clone(), @@ -665,13 +740,13 @@ impl<'a> DeltaScanBuilder<'a> { // Sometimes (i.e Merge) we want to prune files that don't make the // filter and read the entire contents for files that do match the // filter - if let Some(predicate) = logical_filter { + if let Some(predicate) = pushdown_filter { if config.enable_parquet_pushdown { file_source = file_source.with_predicate(Arc::clone(&file_schema), predicate); } }; - let file_scan_config = FileScanConfig::new( + let file_scan_config = FileScanConfigBuilder::new( self.log_store.object_store_url(), file_schema, Arc::new(file_source), @@ -682,15 +757,16 @@ impl<'a> DeltaScanBuilder<'a> { // // See https://github.com/apache/datafusion/issues/11322 if file_groups.is_empty() { - vec![vec![]] + vec![FileGroup::from(vec![])] } else { - file_groups.into_values().collect() + file_groups.into_values().map(FileGroup::from).collect() }, ) .with_statistics(stats) .with_projection(self.projection.cloned()) .with_limit(self.limit) - .with_table_partition_cols(table_partition_cols); + .with_table_partition_cols(table_partition_cols) + .build(); let metrics = ExecutionPlanMetricsSet::new(); MetricBuilder::new(&metrics) @@ -702,7 +778,7 @@ impl<'a> DeltaScanBuilder<'a> { Ok(DeltaScan { table_uri: ensure_table_uri(self.log_store.root_uri())?.as_str().into(), - parquet_scan: file_scan_config.build(), + parquet_scan: DataSourceExec::from_data_source(file_scan_config), config, logical_schema, metrics, @@ -710,6 +786,41 @@ impl<'a> DeltaScanBuilder<'a> { } } +fn simplify_expr( + context: &SessionContext, + df_schema: &DFSchema, + expr: Expr, +) -> Arc { + // Simplify the expression first + let props = ExecutionProps::new(); + let simplify_context = SimplifyContext::new(&props).with_schema(df_schema.clone().into()); + let simplifier = ExprSimplifier::new(simplify_context).with_max_cycles(10); + let simplified = simplifier.simplify(expr).unwrap(); + + context.create_physical_expr(simplified, df_schema).unwrap() +} + +fn prune_file_statistics( + record_batches: &Vec, + pruning_mask: Vec, +) -> Vec { + let mut filtered_batches = Vec::new(); + let mut mask_offset = 0; + + for batch in record_batches { + let num_rows = batch.num_rows(); + let batch_mask = &pruning_mask[mask_offset..mask_offset + num_rows]; + mask_offset += num_rows; + + let boolean_mask = BooleanArray::from(batch_mask.to_vec()); + let filtered_batch = + filter_record_batch(batch, &boolean_mask).expect("Failed to filter RecordBatch"); + filtered_batches.push(filtered_batch); + } + + filtered_batches +} + // TODO: implement this for Snapshot, not for DeltaTable #[async_trait] impl TableProvider for DeltaTable { @@ -757,10 +868,8 @@ impl TableProvider for DeltaTable { &self, filter: &[&Expr], ) -> DataFusionResult> { - Ok(filter - .iter() - .map(|_| TableProviderFilterPushDown::Inexact) - .collect()) + let partition_cols = self.snapshot()?.metadata().partition_columns.as_slice(); + Ok(get_pushdown_filters(filter, partition_cols)) } fn statistics(&self) -> Option { @@ -768,6 +877,72 @@ impl TableProvider for DeltaTable { } } +fn get_pushdown_filters( + filter: &[&Expr], + partition_cols: &[String], +) -> Vec { + filter + .iter() + .cloned() + .map(|expr| { + let applicable = expr_is_exact_predicate_for_cols(partition_cols, expr); + if !expr.column_refs().is_empty() && applicable { + TableProviderFilterPushDown::Exact + } else { + TableProviderFilterPushDown::Inexact + } + }) + .collect() +} + +// inspired from datafusion::listing::helpers, but adapted to only stats based pruning +fn expr_is_exact_predicate_for_cols(partition_cols: &[String], expr: &Expr) -> bool { + let mut is_applicable = true; + expr.apply(|expr| match expr { + Expr::Column(Column { ref name, .. }) => { + is_applicable &= partition_cols.contains(name); + + // TODO: decide if we should constrain this to Utf8 columns (including views, dicts etc) + + if is_applicable { + Ok(TreeNodeRecursion::Jump) + } else { + Ok(TreeNodeRecursion::Stop) + } + } + Expr::BinaryExpr(BinaryExpr { ref op, .. }) => { + is_applicable &= matches!( + op, + Operator::And + | Operator::Or + | Operator::NotEq + | Operator::Eq + | Operator::Gt + | Operator::GtEq + | Operator::Lt + | Operator::LtEq + ); + if is_applicable { + Ok(TreeNodeRecursion::Continue) + } else { + Ok(TreeNodeRecursion::Stop) + } + } + Expr::Literal(_) + | Expr::Not(_) + | Expr::IsNotNull(_) + | Expr::IsNull(_) + | Expr::Between(_) + | Expr::InList(_) => Ok(TreeNodeRecursion::Continue), + _ => { + is_applicable = false; + Ok(TreeNodeRecursion::Stop) + } + }) + .unwrap(); + is_applicable +} + /// A Delta table provider that enables additional metadata columns to be included during the scan #[derive(Debug)] pub struct DeltaTableProvider { @@ -849,10 +1024,8 @@ impl TableProvider for DeltaTableProvider { &self, filter: &[&Expr], ) -> DataFusionResult> { - Ok(filter - .iter() - .map(|_| TableProviderFilterPushDown::Inexact) - .collect()) + let partition_cols = self.snapshot.metadata().partition_columns.as_slice(); + Ok(get_pushdown_filters(filter, partition_cols)) } fn statistics(&self) -> Option { @@ -860,107 +1033,6 @@ impl TableProvider for DeltaTableProvider { } } -#[derive(Debug)] -pub struct LazyTableProvider { - schema: Arc, - batches: Vec>>, -} - -impl LazyTableProvider { - /// Build a DeltaTableProvider - pub fn try_new( - schema: Arc, - batches: Vec>>, - ) -> DeltaResult { - Ok(LazyTableProvider { schema, batches }) - } -} - -#[async_trait] -impl TableProvider for LazyTableProvider { - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> Arc { - self.schema.clone() - } - - fn table_type(&self) -> TableType { - TableType::Base - } - - fn get_table_definition(&self) -> Option<&str> { - None - } - - fn get_logical_plan(&self) -> Option> { - None - } - - async fn scan( - &self, - _session: &dyn Session, - projection: Option<&Vec>, - filters: &[Expr], - limit: Option, - ) -> DataFusionResult> { - let mut plan: Arc = Arc::new(LazyMemoryExec::try_new( - self.schema(), - self.batches.clone(), - )?); - - let df_schema: DFSchema = plan.schema().try_into()?; - - if let Some(filter_expr) = conjunction(filters.iter().cloned()) { - let physical_expr = - create_physical_expr(&filter_expr, &df_schema, &ExecutionProps::new())?; - plan = Arc::new(FilterExec::try_new(physical_expr, plan)?); - } - - if let Some(projection) = projection { - let current_projection = (0..plan.schema().fields().len()).collect::>(); - if projection != ¤t_projection { - let execution_props = &ExecutionProps::new(); - let fields: DeltaResult, String)>> = projection - .iter() - .map(|i| { - let (table_ref, field) = df_schema.qualified_field(*i); - create_physical_expr( - &Expr::Column(Column::from((table_ref, field))), - &df_schema, - execution_props, - ) - .map(|expr| (expr, field.name().clone())) - .map_err(DeltaTableError::from) - }) - .collect(); - plan = Arc::new(ProjectionExec::try_new(fields?, plan)?); - } - } - - if let Some(limit) = limit { - plan = Arc::new(GlobalLimitExec::new(plan, 0, Some(limit))) - }; - - Ok(plan) - } - - fn supports_filters_pushdown( - &self, - filter: &[&Expr], - ) -> DataFusionResult> { - Ok(filter - .iter() - .map(|_| TableProviderFilterPushDown::Inexact) - .collect()) - } - - fn statistics(&self) -> Option { - None - } -} - // TODO: this will likely also need to perform column mapping later when we support reader protocol v2 /// A wrapper for parquet scans #[derive(Debug)] @@ -1415,7 +1487,7 @@ impl DeltaDataChecker { return Ok(()); } let table = MemTable::try_new(record_batch.schema(), vec![vec![record_batch.clone()]])?; - + table.schema(); // Use a random table name to avoid clashes when running multiple parallel tasks, e.g. when using a partitioned table let table_name: String = uuid::Uuid::new_v4().to_string(); self.ctx.register_table(&table_name, Arc::new(table))?; @@ -1425,7 +1497,8 @@ impl DeltaDataChecker { for check in checks { if check.get_name().contains('.') { return Err(DeltaTableError::Generic( - "Support for nested columns is not supported.".to_string(), + "delta constraints for nested columns are not supported at the moment." + .to_string(), )); } @@ -1963,10 +2036,9 @@ impl From for DeltaColumn { #[cfg(test)] mod tests { - use crate::kernel::log_segment::PathExt; use crate::logstore::default_logstore::DefaultLogStore; + use crate::logstore::ObjectStoreRef; use crate::operations::write::SchemaMode; - use crate::storage::ObjectStoreRef; use crate::writer::test_utils::get_delta_schema; use arrow::array::StructArray; use arrow::datatypes::{Field, Schema}; @@ -1974,12 +2046,14 @@ mod tests { use bytes::Bytes; use chrono::{TimeZone, Utc}; use datafusion::assert_batches_sorted_eq; + use datafusion::datasource::physical_plan::FileScanConfig; use datafusion::datasource::source::DataSourceExec; use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::{visit_execution_plan, ExecutionPlanVisitor, PhysicalExpr}; use datafusion_expr::lit; use datafusion_proto::physical_plan::AsExecutionPlan; use datafusion_proto::protobuf; + use delta_kernel::path::{LogPathFileType, ParsedLogPath}; use futures::{stream::BoxStream, StreamExt}; use object_store::{ path::Path, GetOptions, GetResult, ListResult, MultipartUpload, ObjectStore, @@ -2079,7 +2153,6 @@ mod tests { data_change: true, stats: None, deletion_vector: None, - stats_parsed: None, tags: None, base_row_id: None, default_row_commit_version: None, @@ -2232,6 +2305,58 @@ mod tests { assert!(result.is_err()); } + /// Ensure that constraints when there are spaces in the field name still work + /// + /// See + #[tokio::test] + async fn test_constraints_with_spacey_fields() -> DeltaResult<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", ArrowDataType::Utf8, false), + Field::new("b bop", ArrowDataType::Int32, false), + ])); + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(arrow::array::StringArray::from(vec![ + "a", "b bop", "c", "d", + ])), + Arc::new(arrow::array::Int32Array::from(vec![1, 10, 10, 100])), + ], + )?; + + // Valid invariants return Ok(()) + let constraints = vec![ + Constraint::new("custom a", "a is not null"), + Constraint::new("custom_b", "`b bop` < 1000"), + ]; + assert!(DeltaDataChecker::new_with_constraints(constraints) + .check_batch(&batch) + .await + .is_ok()); + + // Violated invariants returns an error with list of violations + let constraints = vec![ + Constraint::new("custom_a", "a is null"), + Constraint::new("custom_B", "\"b bop\" < 100"), + ]; + let result = DeltaDataChecker::new_with_constraints(constraints) + .check_batch(&batch) + .await; + assert!(result.is_err()); + assert!(matches!(result, Err(DeltaTableError::InvalidData { .. }))); + if let Err(DeltaTableError::InvalidData { violations }) = result { + assert_eq!(violations.len(), 2); + } + + // Irrelevant constraints return a different error + let constraints = vec![Constraint::new("custom_c", "c > 2000")]; + let result = DeltaDataChecker::new_with_constraints(constraints) + .check_batch(&batch) + .await; + assert!(result.is_err()); + Ok(()) + } + #[test] fn roundtrip_test_delta_exec_plan() { let ctx = SessionContext::new(); @@ -2306,7 +2431,7 @@ mod tests { .with_partition_columns(["modified", "id"]) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let batch = RecordBatch::try_new( schema.clone(), @@ -2731,10 +2856,6 @@ mod tests { visit_execution_plan(&scan, &mut visitor).unwrap(); assert_eq!(visitor.predicate.unwrap().to_string(), "a@0 = s"); - assert_eq!( - visitor.pruning_predicate.unwrap().orig_expr().to_string(), - "a@0 = s" - ); } #[tokio::test] @@ -2766,7 +2887,6 @@ mod tests { visit_execution_plan(&scan, &mut visitor).unwrap(); assert!(visitor.predicate.is_none()); - assert!(visitor.pruning_predicate.is_none()); } #[tokio::test] @@ -2801,7 +2921,6 @@ mod tests { #[derive(Default)] struct ParquetVisitor { predicate: Option>, - pruning_predicate: Option>, options: Option, } @@ -2828,7 +2947,6 @@ mod tests { { self.options = Some(parquet_source.table_parquet_options().clone()); self.predicate = parquet_source.predicate().cloned(); - self.pruning_predicate = parquet_source.pruning_predicate().cloned(); } Ok(true) @@ -2910,8 +3028,13 @@ mod tests { let (object_store, mut operations) = RecordingObjectStore::new(table.log_store().object_store(None)); - let log_store = - DefaultLogStore::new(Arc::new(object_store), table.log_store().config().clone()); + // this uses an in memory store pointing at root... + let both_store = Arc::new(object_store); + let log_store = DefaultLogStore::new( + both_store.clone(), + both_store, + table.log_store().config().clone(), + ); let provider = DeltaTableProvider::try_new( table.snapshot().unwrap().clone(), Arc::new(log_store), @@ -2974,8 +3097,8 @@ mod tests { #[derive(Debug, PartialEq)] enum ObjectStoreOperation { - GetRanges(LocationType, Vec>), - GetRange(LocationType, Range), + GetRanges(LocationType, Vec>), + GetRange(LocationType, Range), GetOpts(LocationType), Get(LocationType), } @@ -2988,9 +3111,14 @@ mod tests { impl From<&Path> for LocationType { fn from(value: &Path) -> Self { - if value.is_commit_file() { - LocationType::Commit - } else if value.to_string().starts_with("part-") { + let dummy_url = Url::parse("dummy:///").unwrap(); + let parsed = ParsedLogPath::try_from(dummy_url.join(value.as_ref()).unwrap()).unwrap(); + if let Some(parsed) = parsed { + if matches!(parsed.file_type, LogPathFileType::Commit) { + return LocationType::Commit; + } + } + if value.to_string().starts_with("part-") { LocationType::Data } else { panic!("Unknown location type: {value:?}") @@ -3054,7 +3182,7 @@ mod tests { async fn get_range( &self, location: &Path, - range: Range, + range: Range, ) -> object_store::Result { self.operations .send(ObjectStoreOperation::GetRange( @@ -3068,7 +3196,7 @@ mod tests { async fn get_ranges( &self, location: &Path, - ranges: &[Range], + ranges: &[Range], ) -> object_store::Result> { self.operations .send(ObjectStoreOperation::GetRanges( @@ -3094,7 +3222,10 @@ mod tests { self.inner.delete_stream(locations) } - fn list(&self, prefix: Option<&Path>) -> BoxStream<'_, object_store::Result> { + fn list( + &self, + prefix: Option<&Path>, + ) -> BoxStream<'static, object_store::Result> { self.inner.list(prefix) } @@ -3102,7 +3233,7 @@ mod tests { &self, prefix: Option<&Path>, offset: &Path, - ) -> BoxStream<'_, object_store::Result> { + ) -> BoxStream<'static, object_store::Result> { self.inner.list_with_offset(prefix, offset) } diff --git a/crates/core/src/delta_datafusion/planner.rs b/crates/core/src/delta_datafusion/planner.rs index c167b4bb7c..c0eef2b6c0 100644 --- a/crates/core/src/delta_datafusion/planner.rs +++ b/crates/core/src/delta_datafusion/planner.rs @@ -6,7 +6,7 @@ //! #[derive(Clone)] //! struct MergeMetricExtensionPlanner {} //! -//! #[async_trait] +//! #[macro@async_trait] //! impl ExtensionPlanner for MergeMetricExtensionPlanner { //! async fn plan_extension( //! &self, diff --git a/crates/core/src/delta_datafusion/schema_adapter.rs b/crates/core/src/delta_datafusion/schema_adapter.rs index a792c2d285..c9278946b0 100644 --- a/crates/core/src/delta_datafusion/schema_adapter.rs +++ b/crates/core/src/delta_datafusion/schema_adapter.rs @@ -59,7 +59,6 @@ impl SchemaAdapter for DeltaSchemaAdapter { Ok(( Arc::new(SchemaMapping { projected_schema: self.projected_table_schema.clone(), - table_schema: self.table_schema.clone(), }), projection, )) @@ -69,7 +68,6 @@ impl SchemaAdapter for DeltaSchemaAdapter { #[derive(Debug)] pub(crate) struct SchemaMapping { projected_schema: SchemaRef, - table_schema: SchemaRef, } impl SchemaMapper for SchemaMapping { @@ -77,21 +75,4 @@ impl SchemaMapper for SchemaMapping { let record_batch = cast_record_batch(&batch, self.projected_schema.clone(), false, true)?; Ok(record_batch) } - - fn map_partial_batch(&self, batch: RecordBatch) -> datafusion_common::Result { - let partial_table_schema = Arc::new(Schema::new( - batch - .schema() - .fields() - .iter() - .filter_map(|batch_field| { - self.table_schema.field_with_name(batch_field.name()).ok() - }) - .cloned() - .collect::>(), - )); - - let record_batch = cast_record_batch(&batch, partial_table_schema, false, true)?; - Ok(record_batch) - } } diff --git a/crates/core/src/errors.rs b/crates/core/src/errors.rs index 9980cf23ad..dd776dcede 100644 --- a/crates/core/src/errors.rs +++ b/crates/core/src/errors.rs @@ -2,11 +2,10 @@ use chrono::{DateTime, Utc}; use object_store::Error as ObjectStoreError; -use crate::operations::transaction::{CommitBuilderError, TransactionError}; -use crate::protocol::ProtocolError; +use crate::kernel::transaction::{CommitBuilderError, TransactionError}; /// A result returned by delta-rs -pub type DeltaResult = Result; +pub type DeltaResult = Result; /// Delta Table specific error #[allow(missing_docs)] @@ -15,9 +14,6 @@ pub enum DeltaTableError { #[error("Kernel error: {0}")] KernelError(#[from] delta_kernel::error::Error), - #[error("Delta protocol violation: {source}")] - Protocol { source: ProtocolError }, - /// Error returned when reading the delta log object failed. #[error("Failed to read delta log object: {}", .source)] ObjectStore { @@ -249,18 +245,6 @@ impl From for DeltaTableError { } } -impl From for DeltaTableError { - fn from(value: ProtocolError) -> Self { - match value { - ProtocolError::Arrow { source } => DeltaTableError::Arrow { source }, - ProtocolError::IO { source } => DeltaTableError::Io { source }, - ProtocolError::ObjectStore { source } => DeltaTableError::ObjectStore { source }, - ProtocolError::ParquetParseError { source } => DeltaTableError::Parquet { source }, - _ => DeltaTableError::Protocol { source: value }, - } - } -} - impl From for DeltaTableError { fn from(value: serde_json::Error) -> Self { DeltaTableError::InvalidStatsJson { json_err: value } diff --git a/crates/core/src/kernel/arrow/engine_ext.rs b/crates/core/src/kernel/arrow/engine_ext.rs new file mode 100644 index 0000000000..b17be4fc6b --- /dev/null +++ b/crates/core/src/kernel/arrow/engine_ext.rs @@ -0,0 +1,141 @@ +//! Utilities for interacting with Kernel APIs using Arrow data structures. +//! +use delta_kernel::arrow::array::BooleanArray; +use delta_kernel::arrow::compute::filter_record_batch; +use delta_kernel::arrow::record_batch::RecordBatch; +use delta_kernel::engine::arrow_data::ArrowEngineData; +use delta_kernel::scan::{Scan, ScanMetadata}; +use delta_kernel::{ + DeltaResult, Engine, EngineData, ExpressionEvaluator, ExpressionRef, PredicateRef, Version, +}; +use itertools::Itertools; + +/// [`ScanMetadata`] contains (1) a [`RecordBatch`] specifying data files to be scanned +/// and (2) a vector of transforms (one transform per scan file) that must be applied to the data read +/// from those files. +pub(crate) struct ScanMetadataArrow { + /// Record batch with one row per file to scan + pub scan_files: RecordBatch, + + /// Row-level transformations to apply to data read from files. + /// + /// Each entry in this vector corresponds to a row in the `scan_files` data. The entry is an + /// expression that must be applied to convert the file's data into the logical schema + /// expected by the scan: + /// + /// - `Some(expr)`: Apply this expression to transform the data to match [`Scan::schema()`]. + /// - `None`: No transformation is needed; the data is already in the correct logical form. + /// + /// Note: This vector can be indexed by row number. + pub scan_file_transforms: Vec>, +} + +pub(crate) trait ScanExt { + /// Get the metadata for a table scan. + /// + /// This method handles translation between `EngineData` and `RecordBatch` + /// and will already apply any selection vectors to the data. + /// See [`Scan::scan_metadata`] for details. + fn scan_metadata_arrow( + &self, + engine: &dyn Engine, + ) -> DeltaResult>>; + + fn scan_metadata_from_arrow( + &self, + engine: &dyn Engine, + existing_version: Version, + existing_data: Box>, + existing_predicate: Option, + ) -> DeltaResult>>; +} + +impl ScanExt for Scan { + fn scan_metadata_arrow( + &self, + engine: &dyn Engine, + ) -> DeltaResult>> { + Ok(self + .scan_metadata(engine)? + .map_ok(kernel_to_arrow) + .flatten()) + } + + fn scan_metadata_from_arrow( + &self, + engine: &dyn Engine, + existing_version: Version, + existing_data: Box>, + existing_predicate: Option, + ) -> DeltaResult>> { + let engine_iter = + existing_data.map(|batch| Box::new(ArrowEngineData::new(batch)) as Box); + Ok(self + .scan_metadata_from(engine, existing_version, engine_iter, existing_predicate)? + .map_ok(kernel_to_arrow) + .flatten()) + } +} + +fn kernel_to_arrow(metadata: ScanMetadata) -> DeltaResult { + let scan_file_transforms = metadata + .scan_file_transforms + .into_iter() + .enumerate() + .filter_map(|(i, v)| metadata.scan_files.selection_vector[i].then_some(v)) + .collect(); + let batch = ArrowEngineData::try_from_engine_data(metadata.scan_files.data)?.into(); + let scan_files = filter_record_batch( + &batch, + &BooleanArray::from(metadata.scan_files.selection_vector), + )?; + Ok(ScanMetadataArrow { + scan_files, + scan_file_transforms, + }) +} + +pub(crate) trait ExpressionEvaluatorExt { + fn evaluate_arrow(&self, batch: RecordBatch) -> DeltaResult; +} + +impl ExpressionEvaluatorExt for T { + fn evaluate_arrow(&self, batch: RecordBatch) -> DeltaResult { + let engine_data = ArrowEngineData::new(batch); + Ok(ArrowEngineData::try_from_engine_data(T::evaluate(self, &engine_data)?)?.into()) + } +} + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use super::ExpressionEvaluatorExt as _; + + use delta_kernel::arrow::array::Int32Array; + use delta_kernel::arrow::datatypes::{DataType, Field, Schema}; + use delta_kernel::arrow::record_batch::RecordBatch; + use delta_kernel::engine::arrow_conversion::TryIntoKernel; + use delta_kernel::engine::arrow_expression::ArrowEvaluationHandler; + use delta_kernel::expressions::*; + use delta_kernel::EvaluationHandler; + + #[test] + fn test_evaluate_arrow() { + let handler = ArrowEvaluationHandler; + + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + let values = Int32Array::from(vec![1, 2, 3]); + let batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![Arc::new(values)]).unwrap(); + + let expression = column_expr!("a"); + let expr = handler.new_expression_evaluator( + Arc::new((&schema).try_into_kernel().unwrap()), + expression, + delta_kernel::schema::DataType::INTEGER, + ); + + let result = expr.evaluate_arrow(batch); + assert!(result.is_ok()); + } +} diff --git a/crates/core/src/kernel/arrow/mod.rs b/crates/core/src/kernel/arrow/mod.rs index 25cebb85ef..646906f329 100644 --- a/crates/core/src/kernel/arrow/mod.rs +++ b/crates/core/src/kernel/arrow/mod.rs @@ -1,987 +1,4 @@ //! Conversions between Delta and Arrow data types - -use std::sync::{Arc, LazyLock}; - -use arrow_schema::{ - DataType as ArrowDataType, Field as ArrowField, FieldRef as ArrowFieldRef, - Schema as ArrowSchema, SchemaRef as ArrowSchemaRef, -}; - +pub(crate) mod engine_ext; pub(crate) mod extract; pub(crate) mod json; - -const MAP_ROOT_DEFAULT: &str = "key_value"; -const MAP_KEY_DEFAULT: &str = "key"; -const MAP_VALUE_DEFAULT: &str = "value"; - -macro_rules! arrow_map { - ($fieldname: ident, null) => { - ArrowField::new( - stringify!($fieldname), - ArrowDataType::Map( - Arc::new(ArrowField::new( - MAP_ROOT_DEFAULT, - ArrowDataType::Struct( - vec![ - ArrowField::new(MAP_KEY_DEFAULT, ArrowDataType::Utf8, false), - ArrowField::new(MAP_VALUE_DEFAULT, ArrowDataType::Utf8, true), - ] - .into(), - ), - false, - )), - false, - ), - true, - ) - }; - ($fieldname: ident, not_null) => { - ArrowField::new( - stringify!($fieldname), - ArrowDataType::Map( - Arc::new(ArrowField::new( - MAP_ROOT_DEFAULT, - ArrowDataType::Struct( - vec![ - ArrowField::new(MAP_KEY_DEFAULT, ArrowDataType::Utf8, false), - ArrowField::new(MAP_VALUE_DEFAULT, ArrowDataType::Utf8, false), - ] - .into(), - ), - false, - )), - false, - ), - false, - ) - }; -} - -macro_rules! arrow_field { - ($fieldname:ident, $type_qual:ident, null) => { - ArrowField::new(stringify!($fieldname), ArrowDataType::$type_qual, true) - }; - ($fieldname:ident, $type_qual:ident, not_null) => { - ArrowField::new(stringify!($fieldname), ArrowDataType::$type_qual, false) - }; -} - -macro_rules! arrow_list { - ($fieldname:ident, $element_name:ident, $type_qual:ident, null) => { - ArrowField::new( - stringify!($fieldname), - ArrowDataType::List(Arc::new(ArrowField::new( - stringify!($element_name), - ArrowDataType::$type_qual, - true, - ))), - true, - ) - }; - ($fieldname:ident, $element_name:ident, $type_qual:ident, not_null) => { - ArrowField::new( - stringify!($fieldname), - ArrowDataType::List(Arc::new(ArrowField::new( - stringify!($element_name), - ArrowDataType::$type_qual, - true, - ))), - false, - ) - }; -} - -macro_rules! arrow_struct { - ($fieldname:ident, [$($inner:tt)+], null) => { - ArrowField::new( - stringify!($fieldname), - ArrowDataType::Struct( - arrow_defs! [$($inner)+].into() - ), - true - ) - }; - ($fieldname:ident, [$($inner:tt)+], not_null) => { - ArrowField::new( - stringify!($fieldname), - ArrowDataType::Struct( - arrow_defs! [$($inner)+].into() - ), - false - ) - } -} - -macro_rules! arrow_def { - ($fieldname:ident $(null)?) => { - arrow_map!($fieldname, null) - }; - ($fieldname:ident not_null) => { - arrow_map!($fieldname, not_null) - }; - ($fieldname:ident[$inner_name:ident]{$type_qual:ident} $(null)?) => { - arrow_list!($fieldname, $inner_name, $type_qual, null) - }; - ($fieldname:ident[$inner_name:ident]{$type_qual:ident} not_null) => { - arrow_list!($fieldname, $inner_name, $type_qual, not_null) - }; - ($fieldname:ident:$type_qual:ident $(null)?) => { - arrow_field!($fieldname, $type_qual, null) - }; - ($fieldname:ident:$type_qual:ident not_null) => { - arrow_field!($fieldname, $type_qual, not_null) - }; - ($fieldname:ident[$($inner:tt)+] $(null)?) => { - arrow_struct!($fieldname, [$($inner)+], null) - }; - ($fieldname:ident[$($inner:tt)+] not_null) => { - arrow_struct!($fieldname, [$($inner)+], not_null) - } -} - -/// A helper macro to create more readable Arrow field definitions, delimited by commas -/// -/// The argument patterns are as follows: -/// -/// fieldname (null|not_null)? -- An arrow field of type map with name "fieldname" consisting of Utf8 key-value pairs, and an -/// optional nullability qualifier (null if not specified). -/// -/// fieldname:type (null|not_null)? -- An Arrow field consisting of an atomic type. For example, -/// id:Utf8 gets mapped to ArrowField::new("id", ArrowDataType::Utf8, true). -/// where customerCount:Int64 not_null gets mapped to gets mapped to -/// ArrowField::new("customerCount", ArrowDataType::Utf8, true) -/// -/// fieldname[list_element]{list_element_type} (null|not_null)? -- An Arrow list, with the name of the elements wrapped in square brackets -/// and the type of the list elements wrapped in curly brackets. For example, -/// customers[name]{Utf8} is an nullable arrow field of type arrow list consisting -/// of elements called "name" with type Utf8. -/// -/// fieldname[element1, element2, element3, ....] (null|not_null)? -- An arrow struct with name "fieldname" consisting of elements adhering to any of the patterns -/// documented, including additional structs arbitrarily nested up to the recursion -/// limit for Rust macros. -macro_rules! arrow_defs { - () => { - vec![] as Vec - }; - ($($fieldname:ident$(:$type_qual:ident)?$([$($inner:tt)+])?$({$list_type_qual:ident})? $($nullable:ident)?),+) => { - vec![ - $(arrow_def!($fieldname$(:$type_qual)?$([$($inner)+])?$({$list_type_qual})? $($nullable)?)),+ - ] - } -} - -/// Returns an arrow schema representing the delta log for use in checkpoints -/// -/// # Arguments -/// -/// * `table_schema` - The arrow schema representing the table backed by the delta log -/// * `partition_columns` - The list of partition columns of the table. -/// * `use_extended_remove_schema` - Whether to include extended file metadata in remove action schema. -/// Required for compatibility with different versions of Databricks runtime. -pub(crate) fn delta_log_schema_for_table( - table_schema: ArrowSchema, - partition_columns: &[String], - use_extended_remove_schema: bool, - write_stats_as_json: bool, - write_stats_as_struct: bool, -) -> ArrowSchemaRef { - static SCHEMA_FIELDS: LazyLock> = LazyLock::new(|| { - arrow_defs![ - metaData[ - id:Utf8, - name:Utf8, - description:Utf8, - schemaString:Utf8, - createdTime:Int64, - partitionColumns[element]{Utf8}, - configuration, - format[provider:Utf8, options] - ], - protocol[ - minReaderVersion:Int32, - minWriterVersion:Int32, - writerFeatures[element]{Utf8}, - readerFeatures[element]{Utf8} - ], - txn[ - appId:Utf8, - version:Int64 - ] - ] - }); - static ADD_FIELDS: LazyLock> = LazyLock::new(|| { - arrow_defs![ - path:Utf8, - size:Int64, - modificationTime:Int64, - dataChange:Boolean, - partitionValues, - tags, - deletionVector[ - storageType:Utf8 not_null, - pathOrInlineDv:Utf8 not_null, - offset:Int32 null, - sizeInBytes:Int32 not_null, - cardinality:Int64 not_null - ] - ] - }); - static REMOVE_FIELDS: LazyLock> = LazyLock::new(|| { - arrow_defs![ - path: Utf8, - deletionTimestamp: Int64, - dataChange: Boolean, - extendedFileMetadata: Boolean - ] - }); - static REMOVE_EXTENDED_FILE_METADATA_FIELDS: LazyLock> = - LazyLock::new(|| arrow_defs![size: Int64, partitionValues, tags]); - - // create add fields according to the specific data table schema - let (partition_fields, non_partition_fields): (Vec, Vec) = - table_schema - .fields() - .iter() - .map(|field| field.to_owned()) - .partition(|field| partition_columns.contains(field.name())); - let mut add_fields = ADD_FIELDS.clone(); - - if write_stats_as_json { - add_fields.push(ArrowField::new("stats", ArrowDataType::Utf8, true)) - } - - if write_stats_as_struct { - let mut stats_parsed_fields: Vec = - vec![ArrowField::new("numRecords", ArrowDataType::Int64, true)]; - if !non_partition_fields.is_empty() { - let mut max_min_vec = Vec::new(); - non_partition_fields - .iter() - .for_each(|f| max_min_schema_for_fields(&mut max_min_vec, f)); - - if !max_min_vec.is_empty() { - stats_parsed_fields.extend(["minValues", "maxValues"].into_iter().map(|name| { - ArrowField::new( - name, - ArrowDataType::Struct(max_min_vec.clone().into()), - true, - ) - })); - } - - let mut null_count_vec = Vec::new(); - non_partition_fields - .iter() - .for_each(|f| null_count_schema_for_fields(&mut null_count_vec, f)); - let null_count_struct = ArrowField::new( - "nullCount", - ArrowDataType::Struct(null_count_vec.into()), - true, - ); - - stats_parsed_fields.push(null_count_struct); - } - - add_fields.push(ArrowField::new( - "stats_parsed", - ArrowDataType::Struct(stats_parsed_fields.into()), - true, - )); - if !partition_fields.is_empty() { - add_fields.push(ArrowField::new( - "partitionValues_parsed", - ArrowDataType::Struct(partition_fields.into()), - true, - )); - } - } - // create remove fields with or without extendedFileMetadata - let mut remove_fields = REMOVE_FIELDS.clone(); - if use_extended_remove_schema { - remove_fields.extend(REMOVE_EXTENDED_FILE_METADATA_FIELDS.clone()); - } - - // include add and remove fields in checkpoint schema - let mut schema_fields = SCHEMA_FIELDS.clone(); - schema_fields.push(ArrowField::new( - "add", - ArrowDataType::Struct(add_fields.into()), - true, - )); - schema_fields.push(ArrowField::new( - "remove", - ArrowDataType::Struct(remove_fields.into()), - true, - )); - - let arrow_schema = ArrowSchema::new(schema_fields); - - std::sync::Arc::new(arrow_schema) -} - -fn max_min_schema_for_fields(dest: &mut Vec, f: &ArrowField) { - match f.data_type() { - ArrowDataType::Struct(struct_fields) => { - let mut child_dest = Vec::new(); - - for f in struct_fields { - max_min_schema_for_fields(&mut child_dest, f); - } - - if !child_dest.is_empty() { - dest.push(ArrowField::new( - f.name(), - ArrowDataType::Struct(child_dest.into()), - true, - )); - } - } - // don't compute min or max for list, map or binary types - ArrowDataType::List(_) | ArrowDataType::Map(_, _) | ArrowDataType::Binary => { /* noop */ } - _ => { - dest.push(ArrowField::new(f.name(), f.data_type().clone(), true)); - } - } -} - -fn null_count_schema_for_fields(dest: &mut Vec, f: &ArrowField) { - match f.data_type() { - ArrowDataType::Struct(struct_fields) => { - let mut child_dest = Vec::new(); - - for f in struct_fields { - null_count_schema_for_fields(&mut child_dest, f); - } - - dest.push(ArrowField::new( - f.name(), - ArrowDataType::Struct(child_dest.into()), - true, - )); - } - _ => { - let f = ArrowField::new(f.name(), ArrowDataType::Int64, true); - dest.push(f); - } - } -} - -#[cfg(test)] -mod tests { - use std::collections::HashMap; - use std::sync::Arc; - - use arrow::array::ArrayData; - use arrow_array::{Array, BinaryArray, MapArray, RecordBatch, StringArray, StructArray}; - use arrow_buffer::{Buffer, ToByteSlice}; - use delta_kernel::schema::{DataType, MapType, PrimitiveType, StructField, StructType}; - - use super::*; - - #[test] - fn delta_log_schema_for_table_test() { - // NOTE: We should future proof the checkpoint schema in case action schema changes. - // See https://github.com/delta-io/delta-rs/issues/287 - - let table_schema = ArrowSchema::new(vec![ - ArrowField::new("pcol", ArrowDataType::Int32, true), - ArrowField::new("col1", ArrowDataType::Int32, true), - ]); - let partition_columns = vec!["pcol".to_string()]; - let log_schema = delta_log_schema_for_table( - table_schema.clone(), - partition_columns.as_slice(), - false, - true, - true, - ); - - // verify top-level schema contains all expected fields and they are named correctly. - let expected_fields = ["metaData", "protocol", "txn", "remove", "add"]; - for f in log_schema.fields().iter() { - assert!(expected_fields.contains(&f.name().as_str())); - } - assert_eq!(5, log_schema.fields().len()); - - // verify add fields match as expected. a lot of transformation goes into these. - let add_fields: Vec<_> = log_schema - .fields() - .iter() - .filter(|f| f.name() == "add") - .flat_map(|f| { - if let ArrowDataType::Struct(fields) = f.data_type() { - fields.iter().cloned() - } else { - unreachable!(); - } - }) - .collect(); - let field_names: Vec<&String> = add_fields.iter().map(|v| v.name()).collect(); - assert_eq!( - vec![ - "path", - "size", - "modificationTime", - "dataChange", - "partitionValues", - "tags", - "deletionVector", - "stats", - "stats_parsed", - "partitionValues_parsed" - ], - field_names - ); - let add_field_map: HashMap<_, _> = add_fields - .iter() - .map(|f| (f.name().to_owned(), f.clone())) - .collect(); - let partition_values_parsed = add_field_map.get("partitionValues_parsed").unwrap(); - if let ArrowDataType::Struct(fields) = partition_values_parsed.data_type() { - assert_eq!(1, fields.len()); - let field = fields.first().unwrap().to_owned(); - assert_eq!( - Arc::new(ArrowField::new("pcol", ArrowDataType::Int32, true)), - field - ); - } else { - unreachable!(); - } - let stats_parsed = add_field_map.get("stats_parsed").unwrap(); - if let ArrowDataType::Struct(fields) = stats_parsed.data_type() { - assert_eq!(4, fields.len()); - - let field_map: HashMap<_, _> = fields - .iter() - .map(|f| (f.name().to_owned(), f.clone())) - .collect(); - - for (k, v) in field_map.iter() { - match k.as_ref() { - "minValues" | "maxValues" | "nullCount" => match v.data_type() { - ArrowDataType::Struct(fields) => { - assert_eq!(1, fields.len()); - let field = fields.first().unwrap().to_owned(); - let data_type = if k == "nullCount" { - ArrowDataType::Int64 - } else { - ArrowDataType::Int32 - }; - assert_eq!(Arc::new(ArrowField::new("col1", data_type, true)), field); - } - _ => unreachable!(), - }, - "numRecords" => {} - _ => panic!(), - } - } - } else { - unreachable!(); - } - - // verify extended remove schema fields **ARE NOT** included when `use_extended_remove_schema` is false. - let num_remove_fields = log_schema - .fields() - .iter() - .filter(|f| f.name() == "remove") - .flat_map(|f| { - if let ArrowDataType::Struct(fields) = f.data_type() { - fields.iter().cloned() - } else { - unreachable!(); - } - }) - .count(); - assert_eq!(4, num_remove_fields); - - // verify extended remove schema fields **ARE** included when `use_extended_remove_schema` is true. - let log_schema = delta_log_schema_for_table( - table_schema, - partition_columns.as_slice(), - true, - true, - false, - ); - let remove_fields: Vec<_> = log_schema - .fields() - .iter() - .filter(|f| f.name() == "remove") - .flat_map(|f| { - if let ArrowDataType::Struct(fields) = f.data_type() { - fields.iter().cloned() - } else { - unreachable!(); - } - }) - .collect(); - assert_eq!(7, remove_fields.len()); - let expected_fields = [ - "path", - "deletionTimestamp", - "dataChange", - "extendedFileMetadata", - "partitionValues", - "size", - "tags", - ]; - for f in remove_fields.iter() { - assert!(expected_fields.contains(&f.name().as_str())); - } - } - - #[test] - fn delta_log_schema_for_table_test_stats_struct_disabled() { - // NOTE: We should future proof the checkpoint schema in case action schema changes. - // See https://github.com/delta-io/delta-rs/issues/287 - - let table_schema = ArrowSchema::new(vec![ - ArrowField::new("pcol", ArrowDataType::Int32, true), - ArrowField::new("col1", ArrowDataType::Int32, true), - ]); - let partition_columns = vec!["pcol".to_string()]; - let log_schema = delta_log_schema_for_table( - table_schema.clone(), - partition_columns.as_slice(), - false, - true, - false, - ); - - // verify top-level schema contains all expected fields and they are named correctly. - let expected_fields = ["metaData", "protocol", "txn", "remove", "add"]; - for f in log_schema.fields().iter() { - assert!(expected_fields.contains(&f.name().as_str())); - } - assert_eq!(5, log_schema.fields().len()); - - // verify add fields match as expected. a lot of transformation goes into these. - let add_fields: Vec<_> = log_schema - .fields() - .iter() - .filter(|f| f.name() == "add") - .flat_map(|f| { - if let ArrowDataType::Struct(fields) = f.data_type() { - fields.iter().cloned() - } else { - unreachable!(); - } - }) - .collect(); - let field_names: Vec<&String> = add_fields.iter().map(|v| v.name()).collect(); - assert_eq!( - vec![ - "path", - "size", - "modificationTime", - "dataChange", - "partitionValues", - "tags", - "deletionVector", - "stats", - ], - field_names - ); - } - - #[test] - fn delta_log_schema_for_table_test_stats_json_and_struct_disabled() { - // NOTE: We should future proof the checkpoint schema in case action schema changes. - // See https://github.com/delta-io/delta-rs/issues/287 - - let table_schema = ArrowSchema::new(vec![ - ArrowField::new("pcol", ArrowDataType::Int32, true), - ArrowField::new("col1", ArrowDataType::Int32, true), - ]); - let partition_columns = vec!["pcol".to_string()]; - let log_schema = delta_log_schema_for_table( - table_schema.clone(), - partition_columns.as_slice(), - false, - false, - false, - ); - - // verify top-level schema contains all expected fields and they are named correctly. - let expected_fields = ["metaData", "protocol", "txn", "remove", "add"]; - for f in log_schema.fields().iter() { - assert!(expected_fields.contains(&f.name().as_str())); - } - assert_eq!(5, log_schema.fields().len()); - - // verify add fields match as expected. a lot of transformation goes into these. - let add_fields: Vec<_> = log_schema - .fields() - .iter() - .filter(|f| f.name() == "add") - .flat_map(|f| { - if let ArrowDataType::Struct(fields) = f.data_type() { - fields.iter().cloned() - } else { - unreachable!(); - } - }) - .collect(); - let field_names: Vec<&String> = add_fields.iter().map(|v| v.name()).collect(); - assert_eq!( - vec![ - "path", - "size", - "modificationTime", - "dataChange", - "partitionValues", - "tags", - "deletionVector", - ], - field_names - ); - } - - #[test] - fn delta_log_schema_for_table_test_struct_stats_only() { - // NOTE: We should future proof the checkpoint schema in case action schema changes. - // See https://github.com/delta-io/delta-rs/issues/287 - - let table_schema = ArrowSchema::new(vec![ - ArrowField::new("pcol", ArrowDataType::Int32, true), - ArrowField::new("col1", ArrowDataType::Int32, true), - ]); - let partition_columns = vec!["pcol".to_string()]; - let log_schema = delta_log_schema_for_table( - table_schema.clone(), - partition_columns.as_slice(), - false, - false, - true, - ); - - // verify top-level schema contains all expected fields and they are named correctly. - let expected_fields = ["metaData", "protocol", "txn", "remove", "add"]; - for f in log_schema.fields().iter() { - assert!(expected_fields.contains(&f.name().as_str())); - } - assert_eq!(5, log_schema.fields().len()); - - // verify add fields match as expected. a lot of transformation goes into these. - let add_fields: Vec<_> = log_schema - .fields() - .iter() - .filter(|f| f.name() == "add") - .flat_map(|f| { - if let ArrowDataType::Struct(fields) = f.data_type() { - fields.iter().cloned() - } else { - unreachable!(); - } - }) - .collect(); - let field_names: Vec<&String> = add_fields.iter().map(|v| v.name()).collect(); - assert_eq!( - vec![ - "path", - "size", - "modificationTime", - "dataChange", - "partitionValues", - "tags", - "deletionVector", - "stats_parsed", - "partitionValues_parsed" - ], - field_names - ); - let add_field_map: HashMap<_, _> = add_fields - .iter() - .map(|f| (f.name().to_owned(), f.clone())) - .collect(); - let partition_values_parsed = add_field_map.get("partitionValues_parsed").unwrap(); - if let ArrowDataType::Struct(fields) = partition_values_parsed.data_type() { - assert_eq!(1, fields.len()); - let field = fields.first().unwrap().to_owned(); - assert_eq!( - Arc::new(ArrowField::new("pcol", ArrowDataType::Int32, true)), - field - ); - } else { - unreachable!(); - } - let stats_parsed = add_field_map.get("stats_parsed").unwrap(); - if let ArrowDataType::Struct(fields) = stats_parsed.data_type() { - assert_eq!(4, fields.len()); - - let field_map: HashMap<_, _> = fields - .iter() - .map(|f| (f.name().to_owned(), f.clone())) - .collect(); - - for (k, v) in field_map.iter() { - match k.as_ref() { - "minValues" | "maxValues" | "nullCount" => match v.data_type() { - ArrowDataType::Struct(fields) => { - assert_eq!(1, fields.len()); - let field = fields.first().unwrap().to_owned(); - let data_type = if k == "nullCount" { - ArrowDataType::Int64 - } else { - ArrowDataType::Int32 - }; - assert_eq!(Arc::new(ArrowField::new("col1", data_type, true)), field); - } - _ => unreachable!(), - }, - "numRecords" => {} - _ => panic!(), - } - } - } else { - unreachable!(); - } - - // verify extended remove schema fields **ARE NOT** included when `use_extended_remove_schema` is false. - let num_remove_fields = log_schema - .fields() - .iter() - .filter(|f| f.name() == "remove") - .flat_map(|f| { - if let ArrowDataType::Struct(fields) = f.data_type() { - fields.iter().cloned() - } else { - unreachable!(); - } - }) - .count(); - assert_eq!(4, num_remove_fields); - - // verify extended remove schema fields **ARE** included when `use_extended_remove_schema` is true. - let log_schema = delta_log_schema_for_table( - table_schema, - partition_columns.as_slice(), - true, - true, - false, - ); - let remove_fields: Vec<_> = log_schema - .fields() - .iter() - .filter(|f| f.name() == "remove") - .flat_map(|f| { - if let ArrowDataType::Struct(fields) = f.data_type() { - fields.iter().cloned() - } else { - unreachable!(); - } - }) - .collect(); - assert_eq!(7, remove_fields.len()); - let expected_fields = [ - "path", - "deletionTimestamp", - "dataChange", - "extendedFileMetadata", - "partitionValues", - "size", - "tags", - ]; - for f in remove_fields.iter() { - assert!(expected_fields.contains(&f.name().as_str())); - } - } - - #[test] - fn test_record_batch_from_map_type() { - let keys = vec!["0", "1", "5", "6", "7"]; - let values: Vec<&[u8]> = vec![ - b"test_val_1", - b"test_val_2", - b"long_test_val_3", - b"4", - b"test_val_5", - ]; - let entry_offsets = vec![0u32, 1, 1, 4, 5, 5]; - let num_rows = keys.len(); - - let key_field = Arc::new(ArrowField::new(MAP_KEY_DEFAULT, ArrowDataType::Utf8, false)); - let value_field = Arc::new(ArrowField::new( - MAP_VALUE_DEFAULT, - ArrowDataType::Binary, - false, - )); - let key_value_field = ArrowField::new_struct( - MAP_ROOT_DEFAULT, - vec![key_field.clone(), value_field.clone()], - false, - ); - let key_value_array = StructArray::new( - vec![key_field, value_field].into(), - vec![ - Arc::new(StringArray::from(keys)), - Arc::new(BinaryArray::from(values)), - ], - None, - ); - let entry_offsets_buffer = Buffer::from(entry_offsets.as_slice().to_byte_slice()); - - let map_data_type = ArrowDataType::Map(Arc::new(key_value_field), false); - let map_data = ArrayData::builder(map_data_type) - .len(entry_offsets.len() - 1) - .add_buffer(entry_offsets_buffer) - .add_child_data(key_value_array.into_data()) - .build() - .unwrap(); - - let map_array = MapArray::from(map_data); - - let schema = - >::try_from(&StructType::new(vec![ - StructField::new( - "example".to_string(), - DataType::Map(Box::new(MapType::new( - DataType::Primitive(PrimitiveType::String), - DataType::Primitive(PrimitiveType::Binary), - false, - ))), - false, - ), - ])) - .expect("Could not get schema"); - - let record_batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(map_array)]) - .expect("Failed to create RecordBatch"); - - assert_eq!(record_batch.num_columns(), 1); - assert_eq!(record_batch.num_rows(), num_rows); - } - - #[test] - fn test_max_min_schema_for_fields() { - let mut max_min_vec: Vec = Vec::new(); - let fields = [ - ArrowField::new("simple", ArrowDataType::Int32, true), - ArrowField::new( - "struct", - ArrowDataType::Struct( - vec![ArrowField::new("simple", ArrowDataType::Int32, true)].into(), - ), - true, - ), - ArrowField::new( - "list", - ArrowDataType::List(Arc::new(ArrowField::new( - "simple", - ArrowDataType::Int32, - true, - ))), - true, - ), - ArrowField::new( - "map", - ArrowDataType::Map( - Arc::new(ArrowField::new( - "struct", - ArrowDataType::Struct( - vec![ - ArrowField::new("key", ArrowDataType::Int32, true), - ArrowField::new("value", ArrowDataType::Int32, true), - ] - .into(), - ), - true, - )), - true, - ), - true, - ), - ArrowField::new("binary", ArrowDataType::Binary, true), - ]; - - let expected = vec![fields[0].clone(), fields[1].clone()]; - - fields - .iter() - .for_each(|f| max_min_schema_for_fields(&mut max_min_vec, f)); - - assert_eq!(max_min_vec, expected); - } - - #[test] - fn test_null_count_schema_for_fields() { - let mut null_count_vec: Vec = Vec::new(); - let fields = [ - ArrowField::new("int32", ArrowDataType::Int32, true), - ArrowField::new("int64", ArrowDataType::Int64, true), - ArrowField::new("Utf8", ArrowDataType::Utf8, true), - ArrowField::new( - "list", - ArrowDataType::List(Arc::new(ArrowField::new( - "simple", - ArrowDataType::Int32, - true, - ))), - true, - ), - ArrowField::new( - "map", - ArrowDataType::Map( - Arc::new(ArrowField::new( - "struct", - ArrowDataType::Struct( - vec![ - ArrowField::new("key", ArrowDataType::Int32, true), - ArrowField::new("value", ArrowDataType::Int32, true), - ] - .into(), - ), - true, - )), - true, - ), - true, - ), - ArrowField::new( - "struct", - ArrowDataType::Struct( - vec![ArrowField::new("int32", ArrowDataType::Int32, true)].into(), - ), - true, - ), - ]; - let expected = vec![ - ArrowField::new(fields[0].name(), ArrowDataType::Int64, true), - ArrowField::new(fields[1].name(), ArrowDataType::Int64, true), - ArrowField::new(fields[2].name(), ArrowDataType::Int64, true), - ArrowField::new(fields[3].name(), ArrowDataType::Int64, true), - ArrowField::new(fields[4].name(), ArrowDataType::Int64, true), - ArrowField::new( - fields[5].name(), - ArrowDataType::Struct( - vec![ArrowField::new("int32", ArrowDataType::Int64, true)].into(), - ), - true, - ), - ]; - fields - .iter() - .for_each(|f| null_count_schema_for_fields(&mut null_count_vec, f)); - assert_eq!(null_count_vec, expected); - } - - /* - * This test validates the trait implementation of - * TryFrom<&Arc> for schema::SchemaField which is required with Arrow 37 since - * iterators on Fields will give an &Arc - */ - #[test] - fn tryfrom_arrowfieldref_with_structs() { - let field = Arc::new(ArrowField::new( - "test_struct", - ArrowDataType::Struct( - vec![ - ArrowField::new("key", ArrowDataType::Int32, true), - ArrowField::new("value", ArrowDataType::Int32, true), - ] - .into(), - ), - true, - )); - let _converted: StructField = field.as_ref().try_into().unwrap(); - } -} diff --git a/crates/core/src/kernel/mod.rs b/crates/core/src/kernel/mod.rs index 44a09d7745..6a9ba71c94 100644 --- a/crates/core/src/kernel/mod.rs +++ b/crates/core/src/kernel/mod.rs @@ -2,7 +2,7 @@ //! //! The Kernel module contains all the logic for reading and processing the Delta Lake transaction log. -use delta_kernel::engine::arrow_expression::ArrowExpressionHandler; +use delta_kernel::engine::arrow_expression::ArrowEvaluationHandler; use std::{any::Any, sync::LazyLock}; pub mod arrow; @@ -10,6 +10,7 @@ pub mod error; pub mod models; pub mod scalars; mod snapshot; +pub mod transaction; pub use error::*; pub use models::*; @@ -25,5 +26,5 @@ pub trait DataCheck { fn as_any(&self) -> &dyn Any; } -static ARROW_HANDLER: LazyLock = - LazyLock::new(|| ArrowExpressionHandler {}); +static ARROW_HANDLER: LazyLock = + LazyLock::new(|| ArrowEvaluationHandler {}); diff --git a/crates/core/src/kernel/models/actions.rs b/crates/core/src/kernel/models/actions.rs index a157a1df97..c289ba04c3 100644 --- a/crates/core/src/kernel/models/actions.rs +++ b/crates/core/src/kernel/models/actions.rs @@ -3,16 +3,15 @@ use std::fmt::{self, Display}; use std::str::FromStr; use delta_kernel::schema::{DataType, StructField}; +use delta_kernel::table_features::{ReaderFeature, WriterFeature}; use maplit::hashset; use serde::{Deserialize, Serialize}; use tracing::warn; -use url::Url; use super::schema::StructType; use super::StructTypeExt; use crate::kernel::{error::Error, DeltaResult}; use crate::TableProperty; -use delta_kernel::table_features::{ReaderFeatures, WriterFeatures}; /// Defines a file format used in table #[derive(Serialize, Deserialize, Debug, Clone, PartialEq, Eq)] @@ -136,7 +135,7 @@ pub fn contains_timestampntz<'a>(mut fields: impl Iterator>, + pub reader_features: Option>, /// A collection of features that a client must implement in order to correctly /// write this table (exist only when minWriterVersion is set to 7) #[serde(skip_serializing_if = "Option::is_none")] - pub writer_features: Option>, + pub writer_features: Option>, +} + +impl Default for Protocol { + fn default() -> Self { + Self { + min_reader_version: 1, + min_writer_version: 2, + reader_features: None, + writer_features: None, + } + } } impl Protocol { @@ -170,7 +180,7 @@ impl Protocol { /// Append the reader features in the protocol action, automatically bumps min_reader_version pub fn append_reader_features( mut self, - reader_features: impl IntoIterator>, + reader_features: impl IntoIterator>, ) -> Self { let all_reader_features = reader_features .into_iter() @@ -192,7 +202,7 @@ impl Protocol { /// Append the writer features in the protocol action, automatically bumps min_writer_version pub fn append_writer_features( mut self, - writer_features: impl IntoIterator>, + writer_features: impl IntoIterator>, ) -> Self { let all_writer_features = writer_features .into_iter() @@ -235,27 +245,27 @@ impl Protocol { }) .filter_map(|(key, value)| match key.as_str() { "delta.enableChangeDataFeed" if parse_bool(value) => { - Some(WriterFeatures::ChangeDataFeed) + Some(WriterFeature::ChangeDataFeed) } - "delta.appendOnly" if parse_bool(value) => Some(WriterFeatures::AppendOnly), + "delta.appendOnly" if parse_bool(value) => Some(WriterFeature::AppendOnly), "delta.enableDeletionVectors" if parse_bool(value) => { - Some(WriterFeatures::DeletionVectors) + Some(WriterFeature::DeletionVectors) } "delta.enableRowTracking" if parse_bool(value) => { - Some(WriterFeatures::RowTracking) + Some(WriterFeature::RowTracking) } "delta.checkpointPolicy" if value.clone().unwrap_or_default() == "v2" => { - Some(WriterFeatures::V2Checkpoint) + Some(WriterFeature::V2Checkpoint) } _ => None, }) - .collect::>(); + .collect::>(); if configuration .keys() .any(|v| v.starts_with("delta.constraints.")) { - converted_writer_features.insert(WriterFeatures::CheckConstraints); + converted_writer_features.insert(WriterFeature::CheckConstraints); } match self.writer_features { @@ -271,14 +281,14 @@ impl Protocol { .iter() .filter_map(|(key, value)| match key.as_str() { "delta.enableDeletionVectors" if parse_bool(value) => { - Some(ReaderFeatures::DeletionVectors) + Some(ReaderFeature::DeletionVectors) } "delta.checkpointPolicy" if value.clone().unwrap_or_default() == "v2" => { - Some(ReaderFeatures::V2Checkpoint) + Some(ReaderFeature::V2Checkpoint) } _ => None, }) - .collect::>(); + .collect::>(); match self.reader_features { Some(mut features) => { features.extend(converted_reader_features); @@ -382,12 +392,12 @@ impl Protocol { if self.min_writer_version >= 7 { match self.writer_features { Some(mut features) => { - features.insert(WriterFeatures::ChangeDataFeed); + features.insert(WriterFeature::ChangeDataFeed); self.writer_features = Some(features); } None => { self.writer_features = - Some(hashset! {WriterFeatures::ChangeDataFeed}) + Some(hashset! {WriterFeature::ChangeDataFeed}) } } } else if self.min_writer_version <= 3 { @@ -407,17 +417,17 @@ impl Protocol { Ok(true) => { let writer_features = match self.writer_features { Some(mut features) => { - features.insert(WriterFeatures::DeletionVectors); + features.insert(WriterFeature::DeletionVectors); features } - None => hashset! {WriterFeatures::DeletionVectors}, + None => hashset! {WriterFeature::DeletionVectors}, }; let reader_features = match self.reader_features { Some(mut features) => { - features.insert(ReaderFeatures::DeletionVectors); + features.insert(ReaderFeature::DeletionVectors); features } - None => hashset! {ReaderFeatures::DeletionVectors}, + None => hashset! {ReaderFeature::DeletionVectors}, }; self.min_reader_version = 3; self.min_writer_version = 7; @@ -440,8 +450,8 @@ impl Protocol { /// Enable timestamp_ntz in the protocol fn enable_timestamp_ntz(mut self) -> Self { - self = self.append_reader_features([ReaderFeatures::TimestampWithoutTimezone]); - self = self.append_writer_features([WriterFeatures::TimestampWithoutTimezone]); + self = self.append_reader_features([ReaderFeature::TimestampWithoutTimezone]); + self = self.append_writer_features([WriterFeature::TimestampWithoutTimezone]); self } @@ -451,7 +461,7 @@ impl Protocol { self.min_writer_version = 4; } if self.min_writer_version >= 7 { - self = self.append_writer_features([WriterFeatures::GeneratedColumns]); + self = self.append_writer_features([WriterFeature::GeneratedColumns]); } self } @@ -459,7 +469,7 @@ impl Protocol { /// Enabled generated columns fn enable_invariants(mut self) -> Self { if self.min_writer_version >= 7 { - self = self.append_writer_features([WriterFeatures::Invariants]); + self = self.append_writer_features([WriterFeature::Invariants]); } self } @@ -547,27 +557,37 @@ impl fmt::Display for TableFeatures { } } -impl TryFrom<&TableFeatures> for ReaderFeatures { +impl TryFrom<&TableFeatures> for ReaderFeature { type Error = strum::ParseError; fn try_from(value: &TableFeatures) -> Result { - ReaderFeatures::try_from(value.as_ref()) + ReaderFeature::try_from(value.as_ref()) } } -impl TryFrom<&TableFeatures> for WriterFeatures { +impl TryFrom<&TableFeatures> for WriterFeature { type Error = strum::ParseError; fn try_from(value: &TableFeatures) -> Result { - WriterFeatures::try_from(value.as_ref()) + WriterFeature::try_from(value.as_ref()) } } impl TableFeatures { /// Convert table feature to respective reader or/and write feature - pub fn to_reader_writer_features(&self) -> (Option, Option) { - let reader_feature = ReaderFeatures::try_from(self).ok(); - let writer_feature = WriterFeatures::try_from(self).ok(); + pub fn to_reader_writer_features(&self) -> (Option, Option) { + let reader_feature = ReaderFeature::try_from(self) + .ok() + .and_then(|feature| match feature { + ReaderFeature::Unknown(_) => None, + _ => Some(feature), + }); + let writer_feature = WriterFeature::try_from(self) + .ok() + .and_then(|feature| match feature { + WriterFeature::Unknown(_) => None, + _ => Some(feature), + }); (reader_feature, writer_feature) } } @@ -658,92 +678,6 @@ pub struct DeletionVectorDescriptor { pub cardinality: i64, } -impl DeletionVectorDescriptor { - /// get the absolute path of the deletion vector - pub fn absolute_path(&self, parent: &Url) -> DeltaResult> { - match &self.storage_type { - StorageType::UuidRelativePath => { - let prefix_len = self.path_or_inline_dv.len() as i32 - 20; - if prefix_len < 0 { - return Err(Error::DeletionVector("Invalid length".to_string())); - } - let decoded = z85::decode(&self.path_or_inline_dv[(prefix_len as usize)..]) - .map_err(|_| Error::DeletionVector("Failed to decode DV uuid".to_string()))?; - let uuid = uuid::Uuid::from_slice(&decoded) - .map_err(|err| Error::DeletionVector(err.to_string()))?; - let mut dv_suffix = format!("deletion_vector_{uuid}.bin"); - if prefix_len > 0 { - dv_suffix = format!( - "{}/{dv_suffix}", - &self.path_or_inline_dv[..(prefix_len as usize)], - ); - } - let dv_path = parent - .join(&dv_suffix) - .map_err(|_| Error::DeletionVector(format!("invalid path: {dv_suffix}")))?; - Ok(Some(dv_path)) - } - StorageType::AbsolutePath => { - Ok(Some(Url::parse(&self.path_or_inline_dv).map_err(|_| { - Error::DeletionVector(format!("invalid path: {}", self.path_or_inline_dv)) - })?)) - } - StorageType::Inline => Ok(None), - } - } - - // TODO read only required byte ranges - // pub fn read( - // &self, - // fs_client: Arc, - // parent: Url, - // ) -> DeltaResult { - // match self.absolute_path(&parent)? { - // None => { - // let bytes = z85::decode(&self.path_or_inline_dv) - // .map_err(|_| Error::DeletionVector("Failed to decode DV".to_string()))?; - // RoaringTreemap::deserialize_from(&bytes[12..]) - // .map_err(|err| Error::DeletionVector(err.to_string())) - // } - // Some(path) => { - // let offset = self.offset; - // let size_in_bytes = self.size_in_bytes; - // - // let dv_data = fs_client - // .read_files(vec![(path, None)])? - // .next() - // .ok_or(Error::MissingData("No deletion Vector data".to_string()))??; - // - // let mut cursor = Cursor::new(dv_data); - // if let Some(offset) = offset { - // // TODO should we read the datasize from the DV file? - // // offset plus datasize bytes - // cursor.set_position((offset + 4) as u64); - // } - // - // let mut buf = vec![0; 4]; - // cursor - // .read(&mut buf) - // .map_err(|err| Error::DeletionVector(err.to_string()))?; - // let magic = - // i32::from_le_bytes(buf.try_into().map_err(|_| { - // Error::DeletionVector("filed to read magic bytes".to_string()) - // })?); - // println!("magic --> : {magic}"); - // // assert!(magic == 1681511377); - // - // let mut buf = vec![0; size_in_bytes as usize]; - // cursor - // .read(&mut buf) - // .map_err(|err| Error::DeletionVector(err.to_string()))?; - // - // RoaringTreemap::deserialize_from(Cursor::new(buf)) - // .map_err(|err| Error::DeletionVector(err.to_string())) - // } - // } - // } -} - #[derive(Serialize, Deserialize, Debug, Clone, Default)] #[serde(rename_all = "camelCase")] /// Defines an add action @@ -791,14 +725,6 @@ pub struct Add { /// The name of the clustering implementation pub clustering_provider: Option, - - /// Contains statistics (e.g., count, min/max values for columns) about the data in this file in - /// raw parquet format. This field needs to be written when statistics are available and the - /// table property: delta.checkpoint.writeStatsAsStruct is set to true. - /// - /// This field is only available in add action records read from checkpoints - #[serde(skip_serializing, skip_deserializing)] - pub stats_parsed: Option, } /// Represents a tombstone (deleted file) in the Delta log. @@ -1130,87 +1056,43 @@ pub(crate) mod serde_path { #[cfg(test)] mod tests { - use std::path::PathBuf; - use super::*; use crate::kernel::PrimitiveType; - fn dv_relateive() -> DeletionVectorDescriptor { - DeletionVectorDescriptor { - storage_type: "u".parse().unwrap(), - path_or_inline_dv: "ab^-aqEH.-t@S}K{vb[*k^".to_string(), - offset: Some(4), - size_in_bytes: 40, - cardinality: 6, - } - } - - fn dv_absolute() -> DeletionVectorDescriptor { - DeletionVectorDescriptor { - storage_type: "p".parse().unwrap(), - path_or_inline_dv: - "s3://mytable/deletion_vector_d2c639aa-8816-431a-aaf6-d3fe2512ff61.bin".to_string(), - offset: Some(4), - size_in_bytes: 40, - cardinality: 6, - } - } - - fn dv_inline() -> DeletionVectorDescriptor { - DeletionVectorDescriptor { - storage_type: "i".parse().unwrap(), - path_or_inline_dv: "wi5b=000010000siXQKl0rr91000f55c8Xg0@@D72lkbi5=-{L".to_string(), - offset: None, - size_in_bytes: 40, - cardinality: 6, - } - } - - fn dv_example() -> DeletionVectorDescriptor { - DeletionVectorDescriptor { - storage_type: "u".parse().unwrap(), - path_or_inline_dv: "vBn[lx{q8@P<9BNH/isA".to_string(), - offset: Some(1), - size_in_bytes: 36, - cardinality: 2, - } - } - - #[test] - fn test_deletion_vector_absolute_path() { - let parent = Url::parse("s3://mytable/").unwrap(); - - let relative = dv_relateive(); - let expected = - Url::parse("s3://mytable/ab/deletion_vector_d2c639aa-8816-431a-aaf6-d3fe2512ff61.bin") - .unwrap(); - assert_eq!(expected, relative.absolute_path(&parent).unwrap().unwrap()); - - let absolute = dv_absolute(); - let expected = - Url::parse("s3://mytable/deletion_vector_d2c639aa-8816-431a-aaf6-d3fe2512ff61.bin") - .unwrap(); - assert_eq!(expected, absolute.absolute_path(&parent).unwrap().unwrap()); - - let inline = dv_inline(); - assert_eq!(None, inline.absolute_path(&parent).unwrap()); - - let path = std::fs::canonicalize(PathBuf::from("../test/tests/data/table-with-dv-small/")) - .unwrap(); - let parent = url::Url::from_directory_path(path).unwrap(); - let dv_url = parent - .join("deletion_vector_61d16c75-6994-46b7-a15b-8b538852e50e.bin") - .unwrap(); - let example = dv_example(); - assert_eq!(dv_url, example.absolute_path(&parent).unwrap().unwrap()); - } - #[test] fn test_primitive() { let types: PrimitiveType = serde_json::from_str("\"string\"").unwrap(); println!("{types:?}"); } + #[test] + fn test_deserialize_protocol() { + // protocol json data + let raw = serde_json::json!( + { + "minReaderVersion": 3, + "minWriterVersion": 7, + "readerFeatures": ["catalogOwned"], + "writerFeatures": ["catalogOwned", "invariants", "appendOnly"] + } + ); + let protocol: Protocol = serde_json::from_value(raw).unwrap(); + assert_eq!(protocol.min_reader_version, 3); + assert_eq!(protocol.min_writer_version, 7); + assert_eq!( + protocol.reader_features, + Some(hashset! {ReaderFeature::Unknown("catalogOwned".to_owned())}) + ); + assert_eq!( + protocol.writer_features, + Some(hashset! { + WriterFeature::Unknown("catalogOwned".to_owned()), + WriterFeature::Invariants, + WriterFeature::AppendOnly + }) + ); + } + // #[test] // fn test_deletion_vector_read() { // let store = Arc::new(LocalFileSystem::new()); diff --git a/crates/core/src/kernel/models/schema.rs b/crates/core/src/kernel/models/schema.rs index 947b6794d1..cf011378fb 100644 --- a/crates/core/src/kernel/models/schema.rs +++ b/crates/core/src/kernel/models/schema.rs @@ -212,10 +212,7 @@ mod tests { let cols = schema.get_generated_columns().unwrap(); assert_eq!(cols.len(), 1); assert_eq!(cols[0].data_type, DataType::INTEGER); - assert_eq!( - cols[0].validation_expr, - "gc = 5 OR (gc IS NULL AND 5 IS NULL)" - ); + assert_eq!(cols[0].validation_expr, "gc <=> 5"); let schema: StructType = serde_json::from_value(json!( { diff --git a/crates/core/src/kernel/scalars.rs b/crates/core/src/kernel/scalars.rs index 51655b4c35..d6cb1719a6 100644 --- a/crates/core/src/kernel/scalars.rs +++ b/crates/core/src/kernel/scalars.rs @@ -5,11 +5,12 @@ use arrow_array::Array; use arrow_schema::TimeUnit; use chrono::{DateTime, TimeZone, Utc}; use delta_kernel::{ + engine::arrow_conversion::TryIntoKernel as _, expressions::{Scalar, StructData}, schema::StructField, }; use object_store::path::Path; -#[cfg(test)] +#[cfg(any(test, feature = "integration_test"))] use serde_json::Value; use urlencoding::encode; @@ -24,7 +25,7 @@ pub trait ScalarExt: Sized { /// Create a [`Scalar`] from an arrow array row fn from_array(arr: &dyn Array, index: usize) -> Option; /// Serialize as serde_json::Value - #[cfg(test)] + #[cfg(any(test, feature = "integration_test"))] fn to_json(&self) -> serde_json::Value; } @@ -48,23 +49,25 @@ impl ScalarExt for Scalar { let date = DateTime::from_timestamp(*days as i64 * 24 * 3600, 0).unwrap(); date.format("%Y-%m-%d").to_string() } - Self::Decimal(value, _, scale) => match scale.cmp(&0) { - Ordering::Equal => value.to_string(), + Self::Decimal(decimal) => match decimal.scale().cmp(&0) { + Ordering::Equal => decimal.bits().to_string(), Ordering::Greater => { - let scalar_multiple = 10_i128.pow(*scale as u32); + let scale = decimal.scale(); + let value = decimal.bits(); + let scalar_multiple = 10_i128.pow(scale as u32); let mut s = String::new(); s.push_str((value / scalar_multiple).to_string().as_str()); s.push('.'); s.push_str(&format!( "{:0>scale$}", value % scalar_multiple, - scale = *scale as usize + scale = scale as usize )); s } Ordering::Less => { - let mut s = value.to_string(); - for _ in 0..*scale { + let mut s = decimal.bits().to_string(); + for _ in 0..decimal.scale() { s.push('0'); } s @@ -74,6 +77,7 @@ impl ScalarExt for Scalar { Self::Null(_) => "null".to_string(), Self::Struct(_) => self.to_string(), Self::Array(_) => self.to_string(), + Self::Map(_) => self.to_string(), } } @@ -94,7 +98,7 @@ impl ScalarExt for Scalar { return None; } if arr.is_null(index) { - return Some(Self::Null(arr.data_type().try_into().ok()?)); + return Some(Self::Null(arr.data_type().try_into_kernel().ok()?)); } match arr.data_type() { @@ -170,12 +174,13 @@ impl ScalarExt for Scalar { .as_any() .downcast_ref::() .map(|v| Self::Double(v.value(index))), - Decimal128(precision, scale) => { - arr.as_any().downcast_ref::().map(|v| { + Decimal128(precision, scale) => arr + .as_any() + .downcast_ref::() + .and_then(|v| { let value = v.value(index); - Self::Decimal(value, *precision, *scale as u8) - }) - } + Self::decimal(value, *precision, *scale as u8).ok() + }), Date32 => arr .as_any() .downcast_ref::() @@ -191,7 +196,7 @@ impl ScalarExt for Scalar { Struct(fields) => { let struct_fields = fields .iter() - .flat_map(|f| TryFrom::try_from(f.as_ref())) + .flat_map(|f| f.as_ref().try_into_kernel()) .collect::>(); let values = arr .as_any() @@ -232,7 +237,7 @@ impl ScalarExt for Scalar { } /// Serializes this scalar as a serde_json::Value. - #[cfg(test)] + #[cfg(any(test, feature = "integration_test"))] fn to_json(&self) -> serde_json::Value { match self { Self::String(s) => Value::String(s.to_owned()), @@ -251,23 +256,25 @@ impl ScalarExt for Scalar { let date = DateTime::from_timestamp(*days as i64 * 24 * 3600, 0).unwrap(); Value::String(date.format("%Y-%m-%d").to_string()) } - Self::Decimal(value, _, scale) => match scale.cmp(&0) { - Ordering::Equal => Value::String(value.to_string()), + Self::Decimal(decimal) => match decimal.scale().cmp(&0) { + Ordering::Equal => Value::String(decimal.bits().to_string()), Ordering::Greater => { - let scalar_multiple = 10_i128.pow(*scale as u32); + let scale = decimal.scale(); + let value = decimal.bits(); + let scalar_multiple = 10_i128.pow(scale as u32); let mut s = String::new(); s.push_str((value / scalar_multiple).to_string().as_str()); s.push('.'); s.push_str(&format!( "{:0>scale$}", value % scalar_multiple, - scale = *scale as usize + scale = scale as usize )); Value::String(s) } Ordering::Less => { - let mut s = value.to_string(); - for _ in 0..*scale { + let mut s = decimal.bits().to_string(); + for _ in 0..decimal.scale() { s.push('0'); } Value::String(s) @@ -277,6 +284,7 @@ impl ScalarExt for Scalar { Self::Null(_) => Value::Null, Self::Struct(_) => unimplemented!(), Self::Array(_) => unimplemented!(), + Self::Map(_) => unimplemented!(), } } } diff --git a/crates/core/src/kernel/snapshot/log_data.rs b/crates/core/src/kernel/snapshot/log_data.rs index dbe8eff3b6..1074b55b26 100644 --- a/crates/core/src/kernel/snapshot/log_data.rs +++ b/crates/core/src/kernel/snapshot/log_data.rs @@ -180,12 +180,12 @@ impl LogicalFile<'_> { /// Datetime of the last modification time of the file. pub fn modification_datetime(&self) -> DeltaResult> { - DateTime::from_timestamp_millis(self.modification_time()).ok_or(DeltaTableError::from( - crate::protocol::ProtocolError::InvalidField(format!( + DateTime::from_timestamp_millis(self.modification_time()).ok_or( + DeltaTableError::MetadataError(format!( "invalid modification_time: {:?}", self.modification_time() )), - )) + ) } /// The partition values for this logical file. @@ -328,7 +328,6 @@ impl LogicalFile<'_> { base_row_id: None, default_row_commit_version: None, clustering_provider: None, - stats_parsed: None, } } @@ -391,7 +390,7 @@ impl<'a> TryFrom<&LogicalFile<'a>> for ObjectMeta { fn try_from(file_stats: &LogicalFile<'a>) -> Result { Ok(ObjectMeta { location: file_stats.object_store_path(), - size: file_stats.size() as usize, + size: file_stats.size() as u64, last_modified: file_stats.modification_datetime()?, version: None, e_tag: None, @@ -562,12 +561,12 @@ mod datafusion { use datafusion_common::scalar::ScalarValue; use datafusion_common::stats::{ColumnStatistics, Precision, Statistics}; use datafusion_common::Column; - use delta_kernel::engine::arrow_data::ArrowEngineData; use delta_kernel::expressions::Expression; use delta_kernel::schema::{DataType, PrimitiveType}; - use delta_kernel::{ExpressionEvaluator, ExpressionHandler}; + use delta_kernel::{EvaluationHandler, ExpressionEvaluator}; use super::*; + use crate::kernel::arrow::engine_ext::ExpressionEvaluatorExt as _; use crate::kernel::arrow::extract::{extract_and_cast_opt, extract_column}; use crate::kernel::ARROW_HANDLER; @@ -795,23 +794,15 @@ mod datafusion { } else { Expression::column(["add", "stats_parsed", stats_field, &column.name]) }; - let evaluator = ARROW_HANDLER.get_evaluator( + let evaluator = ARROW_HANDLER.new_expression_evaluator( crate::kernel::models::fields::log_schema_ref().clone(), expression, field.data_type().clone(), ); let mut results = Vec::with_capacity(self.data.len()); for batch in self.data.iter() { - let engine = ArrowEngineData::new(batch.clone()); - let result = evaluator.evaluate(&engine).ok()?; - let result = result - .any_ref() - .downcast_ref::() - .ok_or(DeltaTableError::generic( - "failed to downcast evaluator result to ArrowEngineData.", - )) - .ok()?; - results.push(result.record_batch().clone()); + let result = evaluator.evaluate_arrow(batch.clone()).ok()?; + results.push(result); } let batch = concat_batches(results[0].schema_ref(), &results).ok()?; batch.column_by_name("output").cloned() @@ -867,7 +858,7 @@ mod datafusion { /// Note: the returned array must contain `num_containers()` rows fn row_counts(&self, _column: &Column) -> Option { static ROW_COUNTS_EVAL: LazyLock> = LazyLock::new(|| { - ARROW_HANDLER.get_evaluator( + ARROW_HANDLER.new_expression_evaluator( crate::kernel::models::fields::log_schema_ref().clone(), Expression::column(["add", "stats_parsed", "numRecords"]), DataType::Primitive(PrimitiveType::Long), @@ -876,29 +867,60 @@ mod datafusion { let mut results = Vec::with_capacity(self.data.len()); for batch in self.data.iter() { - let engine = ArrowEngineData::new(batch.clone()); - let result = ROW_COUNTS_EVAL.evaluate(&engine).ok()?; - let result = result - .any_ref() - .downcast_ref::() - .ok_or(DeltaTableError::generic( - "failed to downcast evaluator result to ArrowEngineData.", - )) - .ok()?; - results.push(result.record_batch().clone()); + let result = ROW_COUNTS_EVAL.evaluate_arrow(batch.clone()).ok()?; + results.push(result); } let batch = concat_batches(results[0].schema_ref(), &results).ok()?; arrow_cast::cast(batch.column_by_name("output")?, &ArrowDataType::UInt64).ok() } - // This function is required since DataFusion 35.0, but is implemented as a no-op - // https://github.com/apache/arrow-datafusion/blob/ec6abece2dcfa68007b87c69eefa6b0d7333f628/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs#L550 - fn contained( - &self, - _column: &Column, - _value: &HashSet, - ) -> Option { - None + // This function is optional but will optimize partition column pruning + fn contained(&self, column: &Column, value: &HashSet) -> Option { + if value.is_empty() || !self.metadata.partition_columns.contains(&column.name) { + return None; + } + + // Retrieve the partition values for the column + let partition_values = self.pick_stats(column, "__dummy__")?; + + let partition_values = partition_values + .as_any() + .downcast_ref::() + .ok_or(DeltaTableError::generic( + "failed to downcast string result to StringArray.", + )) + .ok()?; + + let mut contains = Vec::with_capacity(partition_values.len()); + + // TODO: this was inspired by parquet's BloomFilter pruning, decide if we should + // just convert to Vec for a subset of column types and use .contains + fn check_scalar(pv: &str, value: &ScalarValue) -> bool { + match value { + ScalarValue::Utf8(Some(v)) + | ScalarValue::Utf8View(Some(v)) + | ScalarValue::LargeUtf8(Some(v)) => pv == v, + + ScalarValue::Dictionary(_, inner) => check_scalar(pv, inner), + // FIXME: is this a good enough default or should we sync this with + // expr_applicable_for_cols and bail out with None + _ => value.to_string() == pv, + } + } + + for i in 0..partition_values.len() { + if partition_values.is_null(i) { + contains.push(false); + } else { + contains.push( + value + .iter() + .any(|scalar| check_scalar(partition_values.value(i), scalar)), + ); + } + } + + Some(BooleanArray::from(contains)) } } } diff --git a/crates/core/src/kernel/snapshot/log_segment.rs b/crates/core/src/kernel/snapshot/log_segment.rs index a6b742fde1..2005c0ee8f 100644 --- a/crates/core/src/kernel/snapshot/log_segment.rs +++ b/crates/core/src/kernel/snapshot/log_segment.rs @@ -4,6 +4,8 @@ use std::sync::{Arc, LazyLock}; use arrow_array::RecordBatch; use chrono::Utc; +use delta_kernel::engine::arrow_conversion::TryIntoArrow as _; +use delta_kernel::path::{LogPathFileType, ParsedLogPath}; use futures::{stream::BoxStream, StreamExt, TryStreamExt}; use itertools::Itertools; use object_store::path::Path; @@ -11,93 +13,20 @@ use object_store::{Error as ObjectStoreError, ObjectMeta, ObjectStore}; use parquet::arrow::arrow_reader::{ArrowReaderMetadata, ArrowReaderOptions}; use parquet::arrow::async_reader::{ParquetObjectReader, ParquetRecordBatchStreamBuilder}; use parquet::arrow::ProjectionMask; -use regex::Regex; use serde::{Deserialize, Serialize}; use tracing::debug; +use url::Url; use super::parse; +use crate::kernel::transaction::CommitData; use crate::kernel::{arrow::json, ActionType, Metadata, Protocol, Schema, StructType}; -use crate::logstore::LogStore; -use crate::operations::transaction::CommitData; +use crate::logstore::{LogStore, LogStoreExt}; use crate::{DeltaResult, DeltaTableConfig, DeltaTableError}; const LAST_CHECKPOINT_FILE_NAME: &str = "_last_checkpoint"; - -static CHECKPOINT_FILE_PATTERN: LazyLock = - LazyLock::new(|| Regex::new(r"\d+\.checkpoint(\.\d+\.\d+)?\.parquet").unwrap()); -static DELTA_FILE_PATTERN: LazyLock = LazyLock::new(|| Regex::new(r"^\d+\.json$").unwrap()); -static CRC_FILE_PATTERN: LazyLock = - LazyLock::new(|| Regex::new(r"^(\.\d+(\.crc|\.json)|\d+)\.crc$").unwrap()); -static LAST_CHECKPOINT_FILE_PATTERN: LazyLock = - LazyLock::new(|| Regex::new(r"^_last_checkpoint$").unwrap()); -static LAST_VACUUM_INFO_FILE_PATTERN: LazyLock = - LazyLock::new(|| Regex::new(r"^_last_vacuum_info$").unwrap()); -static DELETION_VECTOR_FILE_PATTERN: LazyLock = - LazyLock::new(|| Regex::new(r".*\.bin$").unwrap()); pub(super) static TOMBSTONE_SCHEMA: LazyLock = LazyLock::new(|| StructType::new(vec![ActionType::Remove.schema_field().clone()])); -/// Trait to extend a file path representation with delta specific functionality -/// -/// specifically, this trait adds the ability to recognize valid log files and -/// parse the version number from a log file path -// TODO handle compaction files -pub(crate) trait PathExt { - /// Returns the last path segment if not terminated with a "/" - fn filename(&self) -> Option<&str>; - - /// Parse the version number assuming a commit json or checkpoint parquet file - fn commit_version(&self) -> Option { - self.filename() - .and_then(|f| f.split_once('.')) - .and_then(|(name, _)| name.parse().ok()) - } - - /// Returns true if the file is a checkpoint parquet file - fn is_checkpoint_file(&self) -> bool { - self.filename() - .map(|name| CHECKPOINT_FILE_PATTERN.captures(name).is_some()) - .unwrap_or(false) - } - - /// Returns true if the file is a commit json file - fn is_commit_file(&self) -> bool { - self.filename() - .map(|name| DELTA_FILE_PATTERN.captures(name).is_some()) - .unwrap_or(false) - } - - fn is_crc_file(&self) -> bool { - self.filename() - .map(|name| CRC_FILE_PATTERN.captures(name).is_some()) - .unwrap() - } - - fn is_last_checkpoint_file(&self) -> bool { - self.filename() - .map(|name| LAST_CHECKPOINT_FILE_PATTERN.captures(name).is_some()) - .unwrap_or(false) - } - - fn is_last_vacuum_info_file(&self) -> bool { - self.filename() - .map(|name| LAST_VACUUM_INFO_FILE_PATTERN.captures(name).is_some()) - .unwrap_or(false) - } - - fn is_deletion_vector_file(&self) -> bool { - self.filename() - .map(|name| DELETION_VECTOR_FILE_PATTERN.captures(name).is_some()) - .unwrap_or(false) - } -} - -impl PathExt for Path { - fn filename(&self) -> Option<&str> { - self.filename() - } -} - #[derive(Debug, Clone, PartialEq)] pub(super) struct LogSegment { pub(super) version: i64, @@ -109,32 +38,40 @@ impl LogSegment { /// Try to create a new [`LogSegment`] /// /// This will list the entire log directory and find all relevant files for the given table version. - pub async fn try_new( - table_root: &Path, - version: Option, - store: &dyn ObjectStore, - ) -> DeltaResult { - let log_url = table_root.child("_delta_log"); - let maybe_cp = read_last_checkpoint(store, &log_url).await?; + pub async fn try_new(log_store: &dyn LogStore, version: Option) -> DeltaResult { + let root_store = log_store.root_object_store(None); + + let root_url = log_store.table_root_url(); + let mut store_root = root_url.clone(); + store_root.set_path(""); + + let log_url = log_store.log_root_url(); + let log_path = crate::logstore::object_store_path(&log_url)?; + + let maybe_cp = read_last_checkpoint(&root_store, &log_path).await?; // List relevant files from log let (mut commit_files, checkpoint_files) = match (maybe_cp, version) { - (Some(cp), None) => list_log_files_with_checkpoint(&cp, store, &log_url).await?, + (Some(cp), None) => { + list_log_files_with_checkpoint(&cp, &root_store, &log_path, &store_root).await? + } (Some(cp), Some(v)) if cp.version <= v => { - list_log_files_with_checkpoint(&cp, store, &log_url).await? + list_log_files_with_checkpoint(&cp, &root_store, &log_path, &store_root).await? } - _ => list_log_files(store, &log_url, version, None).await?, + _ => list_log_files(&root_store, &log_path, version, None, &store_root).await?, }; // remove all files above requested version if let Some(version) = version { - commit_files.retain(|meta| meta.location.commit_version() <= Some(version)); + commit_files.retain(|meta| meta.1.version <= version as u64); } + validate(&commit_files, &checkpoint_files)?; + let mut segment = Self { version: 0, - commit_files: commit_files.into(), - checkpoint_files, + commit_files: commit_files.into_iter().map(|(meta, _)| meta).collect(), + checkpoint_files: checkpoint_files.into_iter().map(|(meta, _)| meta).collect(), }; if segment.commit_files.is_empty() && segment.checkpoint_files.is_empty() { return Err(DeltaTableError::NotATable("no log files".into())); @@ -142,13 +79,11 @@ impl LogSegment { // get the effective version from chosen files let version_eff = segment.file_version().ok_or(DeltaTableError::Generic( "failed to get effective version".into(), - ))?; // TODO: A more descriptive error + ))?; segment.version = version_eff; - segment.validate()?; if let Some(v) = version { if version_eff != v { - // TODO more descriptive error return Err(DeltaTableError::Generic("missing version".into())); } } @@ -161,69 +96,66 @@ impl LogSegment { /// This will create a new [`LogSegment`] from the log with all relevant log files /// starting at `start_version` and ending at `end_version`. pub async fn try_new_slice( - table_root: &Path, start_version: i64, end_version: Option, log_store: &dyn LogStore, ) -> DeltaResult { debug!("try_new_slice: start_version: {start_version}, end_version: {end_version:?}",); log_store.refresh().await?; - let log_url = table_root.child("_delta_log"); + let log_url = log_store.log_root_url(); + let mut store_root = log_url.clone(); + store_root.set_path(""); + let log_path = crate::logstore::object_store_path(&log_url)?; + let (mut commit_files, checkpoint_files) = list_log_files( - log_store.object_store(None).as_ref(), - &log_url, + &log_store.root_object_store(None), + &log_path, end_version, Some(start_version), + &store_root, ) .await?; + // remove all files above requested version if let Some(version) = end_version { - commit_files.retain(|meta| meta.location.commit_version() <= Some(version)); + commit_files.retain(|meta| meta.1.version <= version as u64); } + + validate(&commit_files, &checkpoint_files)?; + let mut segment = Self { version: start_version, - commit_files: commit_files.into(), - checkpoint_files, + commit_files: commit_files.into_iter().map(|(meta, _)| meta).collect(), + checkpoint_files: checkpoint_files.into_iter().map(|(meta, _)| meta).collect(), }; segment.version = segment .file_version() .unwrap_or(end_version.unwrap_or(start_version)); - Ok(segment) - } - pub fn validate(&self) -> DeltaResult<()> { - let checkpoint_version = self - .checkpoint_files - .iter() - .filter_map(|f| f.location.commit_version()) - .max(); - if let Some(v) = checkpoint_version { - if !self - .commit_files - .iter() - .all(|f| f.location.commit_version() > Some(v)) - { - return Err(DeltaTableError::Generic("inconsistent log segment".into())); - } - } - Ok(()) + Ok(segment) } /// Returns the highest commit version number in the log segment pub fn file_version(&self) -> Option { + let dummy_url = Url::parse("dummy:///").unwrap(); self.commit_files - .iter() - .filter_map(|f| f.location.commit_version()) - .max() - .or(self - .checkpoint_files - .first() - .and_then(|f| f.location.commit_version())) + .front() + .and_then(|f| { + let file_url = dummy_url.join(f.location.as_ref()).ok()?; + let parsed = ParsedLogPath::try_from(file_url).ok()?; + parsed.map(|p| p.version as i64) + }) + .or(self.checkpoint_files.first().and_then(|f| { + let file_url = dummy_url.join(f.location.as_ref()).ok()?; + let parsed = ParsedLogPath::try_from(file_url).ok()?; + parsed.map(|p| p.version as i64) + })) } #[cfg(test)] pub(super) fn new_test<'a>( commits: impl IntoIterator, + table_root: &Path, ) -> DeltaResult<(Self, Vec>)> { let mut log = Self { version: -1, @@ -233,7 +165,7 @@ impl LogSegment { let iter = log .advance( commits, - &Path::default(), + table_root, crate::kernel::models::fields::log_schema(), &Default::default(), )? @@ -247,23 +179,33 @@ impl LogSegment { /// Returns the last modified timestamp for a commit file with the given version pub fn version_timestamp(&self, version: i64) -> Option> { + let dummy_url = Url::parse("dummy:///").unwrap(); self.commit_files .iter() - .find(|f| f.location.commit_version() == Some(version)) + .find(|f| { + let parsed = dummy_url + .join(f.location.as_ref()) + .ok() + .and_then(|p| ParsedLogPath::try_from(p).ok()) + .flatten(); + parsed.map(|p| p.version == version as u64).unwrap_or(false) + }) .map(|f| f.last_modified) } pub(super) fn commit_stream( &self, - store: Arc, + log_store: &dyn LogStore, read_schema: &Schema, config: &DeltaTableConfig, ) -> DeltaResult>> { - let decoder = json::get_decoder(Arc::new(read_schema.try_into()?), config)?; + let root_store = log_store.root_object_store(None); + + let decoder = json::get_decoder(Arc::new(read_schema.try_into_arrow()?), config)?; let stream = futures::stream::iter(self.commit_files.iter()) .map(move |meta| { - let store = store.clone(); - async move { store.get(&meta.location).await?.bytes().await } + let root_store = root_store.clone(); + async move { root_store.get(&meta.location).await?.bytes().await } }) .buffered(config.log_buffer_size); Ok(json::decode_stream(decoder, stream).boxed()) @@ -271,18 +213,21 @@ impl LogSegment { pub(super) fn checkpoint_stream( &self, - store: Arc, + log_store: &dyn LogStore, read_schema: &Schema, config: &DeltaTableConfig, ) -> BoxStream<'_, DeltaResult> { + let root_store = log_store.root_object_store(None); + let batch_size = config.log_batch_size; let read_schema = Arc::new(read_schema.clone()); futures::stream::iter(self.checkpoint_files.clone()) .map(move |meta| { - let store = store.clone(); + let root_store = root_store.clone(); let read_schema = read_schema.clone(); async move { - let mut reader = ParquetObjectReader::new(store, meta); + let mut reader = ParquetObjectReader::new(root_store, meta.location) + .with_file_size(meta.size); let options = ArrowReaderOptions::new(); let reader_meta = ArrowReaderMetadata::load_async(&mut reader, options).await?; @@ -322,7 +267,7 @@ impl LogSegment { /// Read [`Protocol`] and [`Metadata`] actions pub(super) async fn read_metadata( &self, - store: Arc, + log_store: &dyn LogStore, config: &DeltaTableConfig, ) -> DeltaResult<(Option, Option)> { static READ_SCHEMA: LazyLock = LazyLock::new(|| { @@ -335,7 +280,7 @@ impl LogSegment { let mut maybe_protocol = None; let mut maybe_metadata = None; - let mut commit_stream = self.commit_stream(store.clone(), &READ_SCHEMA, config)?; + let mut commit_stream = self.commit_stream(log_store, &READ_SCHEMA, config)?; while let Some(batch) = commit_stream.next().await { let batch = batch?; if maybe_protocol.is_none() { @@ -353,7 +298,7 @@ impl LogSegment { } } - let mut checkpoint_stream = self.checkpoint_stream(store.clone(), &READ_SCHEMA, config); + let mut checkpoint_stream = self.checkpoint_stream(log_store, &READ_SCHEMA, config); while let Some(batch) = checkpoint_stream.next().await { let batch = batch?; if maybe_protocol.is_none() { @@ -386,7 +331,7 @@ impl LogSegment { config: &DeltaTableConfig, ) -> DeltaResult> + '_> { let log_path = table_root.child("_delta_log"); - let mut decoder = json::get_decoder(Arc::new(read_schema.try_into()?), config)?; + let mut decoder = json::get_decoder(Arc::new(read_schema.try_into_arrow()?), config)?; let mut commit_data = Vec::new(); for commit in commits { @@ -395,7 +340,7 @@ impl LogSegment { let bytes = commit.get_bytes()?; let meta = ObjectMeta { location: path, - size: bytes.len(), + size: bytes.len() as u64, last_modified: Utc::now(), e_tag: None, version: None, @@ -414,6 +359,30 @@ impl LogSegment { } } +fn validate( + commit_files: &[(ObjectMeta, ParsedLogPath)], + checkpoint_files: &[(ObjectMeta, ParsedLogPath)], +) -> DeltaResult<()> { + let is_contiguous = commit_files + .iter() + .collect_vec() + .windows(2) + .all(|cfs| cfs[0].1.version - 1 == cfs[1].1.version); + if !is_contiguous { + return Err(DeltaTableError::Generic( + "non-contiguous log segment".into(), + )); + } + + let checkpoint_version = checkpoint_files.iter().map(|f| f.1.version).max(); + if let Some(v) = checkpoint_version { + if !commit_files.iter().all(|f| f.1.version > v) { + return Err(DeltaTableError::Generic("inconsistent log segment".into())); + } + } + Ok(()) +} + #[derive(Debug, Deserialize, Serialize)] #[serde(rename_all = "camelCase")] struct CheckpointMetadata { @@ -455,25 +424,32 @@ async fn read_last_checkpoint( /// List all log files after a given checkpoint. async fn list_log_files_with_checkpoint( cp: &CheckpointMetadata, - fs_client: &dyn ObjectStore, + root_store: &dyn ObjectStore, log_root: &Path, -) -> DeltaResult<(Vec, Vec)> { + store_root: &Url, +) -> DeltaResult<( + Vec<(ObjectMeta, ParsedLogPath)>, + Vec<(ObjectMeta, ParsedLogPath)>, +)> { let version_prefix = format!("{:020}", cp.version); let start_from = log_root.child(version_prefix.as_str()); - let files = fs_client + let files = root_store .list_with_offset(Some(log_root), &start_from) .try_collect::>() .await? .into_iter() - // TODO this filters out .crc files etc which start with "." - how do we want to use these kind of files? - .filter(|f| f.location.commit_version().is_some()) + .filter_map(|f| { + let file_url = store_root.join(f.location.as_ref()).ok()?; + let path = ParsedLogPath::try_from(file_url).ok()??; + Some((f, path)) + }) .collect::>(); let mut commit_files = files .iter() .filter_map(|f| { - if f.location.is_commit_file() && f.location.commit_version() > Some(cp.version) { + if matches!(f.1.file_type, LogPathFileType::Commit) && f.1.version > cp.version as u64 { Some(f.clone()) } else { None @@ -482,12 +458,19 @@ async fn list_log_files_with_checkpoint( .collect_vec(); // NOTE: this will sort in reverse order - commit_files.sort_unstable_by(|a, b| b.location.cmp(&a.location)); + commit_files.sort_unstable_by(|a, b| b.0.location.cmp(&a.0.location)); let checkpoint_files = files .iter() .filter_map(|f| { - if f.location.is_checkpoint_file() && f.location.commit_version() == Some(cp.version) { + if matches!( + f.1.file_type, + // UUID named checkpoints are part of the v2 spec and can currently not be parsed. + // This will be supported once we do kernel log replay. + // | LogPathFileType::UuidCheckpoint(_) + LogPathFileType::SinglePartCheckpoint | LogPathFileType::MultiPartCheckpoint { .. } + ) && f.1.version == cp.version as u64 + { Some(f.clone()) } else { None @@ -511,11 +494,15 @@ async fn list_log_files_with_checkpoint( /// /// Relevant files are the max checkpoint found and all subsequent commits. pub(super) async fn list_log_files( - fs_client: &dyn ObjectStore, + root_store: &dyn ObjectStore, log_root: &Path, max_version: Option, start_version: Option, -) -> DeltaResult<(Vec, Vec)> { + store_root: &Url, +) -> DeltaResult<( + Vec<(ObjectMeta, ParsedLogPath)>, + Vec<(ObjectMeta, ParsedLogPath)>, +)> { let max_version = max_version.unwrap_or(i64::MAX - 1); let start_from = log_root.child(format!("{:020}", start_version.unwrap_or(0)).as_str()); @@ -523,19 +510,28 @@ pub(super) async fn list_log_files( let mut commit_files = Vec::with_capacity(25); let mut checkpoint_files = Vec::with_capacity(10); - for meta in fs_client + for meta in root_store .list_with_offset(Some(log_root), &start_from) .try_collect::>() .await? + .into_iter() + .filter_map(|f| { + let file_url = store_root.join(f.location.as_ref()).ok()?; + let path = ParsedLogPath::try_from(file_url).ok()??; + Some((f, path)) + }) { - if meta.location.commit_version().unwrap_or(i64::MAX) <= max_version - && meta.location.commit_version() >= start_version - { - if meta.location.is_checkpoint_file() { - let version = meta.location.commit_version().unwrap_or(0); - match version.cmp(&max_checkpoint_version) { + if meta.1.version <= max_version as u64 && Some(meta.1.version as i64) >= start_version { + if matches!( + meta.1.file_type, + // UUID named checkpoints are part of the v2 spec and can currently not be parsed. + // This will be supported once we do kernel log replay. + // | LogPathFileType::UuidCheckpoint(_) + LogPathFileType::SinglePartCheckpoint | LogPathFileType::MultiPartCheckpoint { .. } + ) { + match (meta.1.version as i64).cmp(&max_checkpoint_version) { Ordering::Greater => { - max_checkpoint_version = version; + max_checkpoint_version = meta.1.version as i64; checkpoint_files.clear(); checkpoint_files.push(meta); } @@ -544,51 +540,49 @@ pub(super) async fn list_log_files( } _ => {} } - } else if meta.location.is_commit_file() { + } else if matches!(meta.1.file_type, LogPathFileType::Commit) { commit_files.push(meta); } } } - commit_files.retain(|f| f.location.commit_version().unwrap_or(0) > max_checkpoint_version); + commit_files.retain(|f| f.1.version as i64 > max_checkpoint_version); // NOTE this will sort in reverse order - commit_files.sort_unstable_by(|a, b| b.location.cmp(&a.location)); + commit_files.sort_unstable_by(|a, b| b.0.location.cmp(&a.0.location)); Ok((commit_files, checkpoint_files)) } #[cfg(test)] pub(super) mod tests { - use delta_kernel::table_features::{ReaderFeatures, WriterFeatures}; - use deltalake_test::utils::*; + use delta_kernel::table_features::{ReaderFeature, WriterFeature}; use maplit::hashset; use tokio::task::JoinHandle; use crate::{ - checkpoints::{create_checkpoint_for, create_checkpoint_from_table_uri_and_cleanup}, + checkpoints::create_checkpoint_from_table_uri_and_cleanup, + kernel::transaction::{CommitBuilder, TableReference}, kernel::{Action, Add, Format, Remove}, - operations::transaction::{CommitBuilder, TableReference}, + protocol::create_checkpoint_for, protocol::{DeltaOperation, SaveMode}, + test_utils::{TestResult, TestTables}, DeltaTableBuilder, }; use super::*; - pub(crate) async fn test_log_segment(context: &IntegrationContext) -> TestResult { - read_log_files(context).await?; - read_metadata(context).await?; - log_segment_serde(context).await?; + pub(crate) async fn test_log_segment() -> TestResult { + read_log_files().await?; + read_metadata().await?; + log_segment_serde().await?; Ok(()) } - async fn log_segment_serde(context: &IntegrationContext) -> TestResult { - let store = context - .table_builder(TestTables::Simple) - .build_storage()? - .object_store(None); + async fn log_segment_serde() -> TestResult { + let log_store = TestTables::Simple.table_builder().build_storage()?; - let segment = LogSegment::try_new(&Path::default(), None, store.as_ref()).await?; + let segment = LogSegment::try_new(&log_store, None).await?; let bytes = serde_json::to_vec(&segment).unwrap(); let actual: LogSegment = serde_json::from_slice(&bytes).unwrap(); assert_eq!(actual.version(), segment.version()); @@ -601,97 +595,97 @@ pub(super) mod tests { Ok(()) } - async fn read_log_files(context: &IntegrationContext) -> TestResult { - let store = context - .table_builder(TestTables::SimpleWithCheckpoint) - .build_storage()? - .object_store(None); - - let log_path = Path::from("_delta_log"); - let cp = read_last_checkpoint(store.as_ref(), &log_path) - .await? - .unwrap(); + async fn read_log_files() -> TestResult { + let log_store = TestTables::SimpleWithCheckpoint + .table_builder() + .build_storage()?; + let root_store = log_store.root_object_store(None); + let log_url = log_store.log_root_url(); + let log_path = Path::from_url_path(log_url.path())?; + let mut store_url = log_url.clone(); + store_url.set_path(""); + + let cp = read_last_checkpoint(&root_store, &log_path).await?.unwrap(); assert_eq!(cp.version, 10); - let (log, check) = list_log_files_with_checkpoint(&cp, store.as_ref(), &log_path).await?; + let (log, check) = + list_log_files_with_checkpoint(&cp, &root_store, &log_path, &store_url).await?; assert_eq!(log.len(), 0); assert_eq!(check.len(), 1); - let (log, check) = list_log_files(store.as_ref(), &log_path, None, None).await?; + let (log, check) = list_log_files(&root_store, &log_path, None, None, &store_url).await?; assert_eq!(log.len(), 0); assert_eq!(check.len(), 1); - let (log, check) = list_log_files(store.as_ref(), &log_path, Some(8), None).await?; + let (log, check) = + list_log_files(&root_store, &log_path, Some(8), None, &store_url).await?; assert_eq!(log.len(), 9); assert_eq!(check.len(), 0); - let segment = LogSegment::try_new(&Path::default(), None, store.as_ref()).await?; + let segment = LogSegment::try_new(&log_store, None).await?; assert_eq!(segment.version, 10); assert_eq!(segment.commit_files.len(), 0); assert_eq!(segment.checkpoint_files.len(), 1); - let segment = LogSegment::try_new(&Path::default(), Some(8), store.as_ref()).await?; + let segment = LogSegment::try_new(&log_store, Some(8)).await?; assert_eq!(segment.version, 8); assert_eq!(segment.commit_files.len(), 9); assert_eq!(segment.checkpoint_files.len(), 0); - let store = context - .table_builder(TestTables::Simple) - .build_storage()? - .object_store(None); + let log_store = TestTables::Simple.table_builder().build_storage()?; + let root_store = log_store.root_object_store(None); + let log_url = log_store.log_root_url(); + let mut store_url = log_url.clone(); + store_url.set_path(""); + let log_path = Path::from_url_path(log_url.path())?; - let (log, check) = list_log_files(store.as_ref(), &log_path, None, None).await?; + let (log, check) = list_log_files(&root_store, &log_path, None, None, &store_url).await?; assert_eq!(log.len(), 5); assert_eq!(check.len(), 0); - let (log, check) = list_log_files(store.as_ref(), &log_path, Some(2), None).await?; + let (log, check) = + list_log_files(&root_store, &log_path, Some(2), None, &store_url).await?; assert_eq!(log.len(), 3); assert_eq!(check.len(), 0); Ok(()) } - async fn read_metadata(context: &IntegrationContext) -> TestResult { - let store = context - .table_builder(TestTables::WithDvSmall) - .build_storage()? - .object_store(None); - let segment = LogSegment::try_new(&Path::default(), None, store.as_ref()).await?; + async fn read_metadata() -> TestResult { + let log_store = TestTables::WithDvSmall.table_builder().build_storage()?; + let segment = LogSegment::try_new(&log_store, None).await?; let (protocol, _metadata) = segment - .read_metadata(store.clone(), &Default::default()) + .read_metadata(&log_store, &Default::default()) .await?; let protocol = protocol.unwrap(); let expected = Protocol { min_reader_version: 3, min_writer_version: 7, - reader_features: Some(hashset! {ReaderFeatures::DeletionVectors}), - writer_features: Some(hashset! {WriterFeatures::DeletionVectors}), + reader_features: Some(hashset! {ReaderFeature::DeletionVectors}), + writer_features: Some(hashset! {WriterFeature::DeletionVectors}), }; assert_eq!(protocol, expected); Ok(()) } - pub(crate) async fn concurrent_checkpoint(context: &IntegrationContext) -> TestResult { - context - .load_table(TestTables::LatestNotCheckpointed) - .await?; - let table_to_checkpoint = context - .table_builder(TestTables::LatestNotCheckpointed) + pub(crate) async fn concurrent_checkpoint() -> TestResult { + let table_to_checkpoint = TestTables::LatestNotCheckpointed + .table_builder() .load() .await?; - let store = context - .table_builder(TestTables::LatestNotCheckpointed) - .build_storage()? - .object_store(None); - let slow_list_store = Arc::new(slow_store::SlowListStore { store }); - let version = table_to_checkpoint.version(); + let base_store = table_to_checkpoint.log_store().root_object_store(None); + let slow_list_store = Arc::new(slow_store::SlowListStore { store: base_store }); + let slow_log_store = TestTables::LatestNotCheckpointed + .table_builder() + .with_storage_backend(slow_list_store, url::Url::parse("dummy:///").unwrap()) + .build_storage()?; + + let version = table_to_checkpoint.version().unwrap(); let load_task: JoinHandle> = tokio::spawn(async move { - let segment = - LogSegment::try_new(&Path::default(), Some(version), slow_list_store.as_ref()) - .await?; + let segment = LogSegment::try_new(&slow_log_store, Some(version)).await?; Ok(segment) }); @@ -759,7 +753,7 @@ pub(super) mod tests { self.store.delete(location).await } - fn list(&self, prefix: Option<&Path>) -> BoxStream<'_, Result> { + fn list(&self, prefix: Option<&Path>) -> BoxStream<'static, Result> { std::thread::sleep(std::time::Duration::from_secs(1)); self.store.list(prefix) } @@ -778,23 +772,6 @@ pub(super) mod tests { } } - #[test] - pub fn is_commit_file_only_matches_commits() { - for path in [0, 1, 5, 10, 100, i64::MAX] - .into_iter() - .map(crate::storage::commit_uri_from_version) - { - assert!(path.is_commit_file()); - } - - let not_commits = ["_delta_log/_commit_2132c4fe-4077-476c-b8f5-e77fea04f170.json.tmp"]; - - for not_commit in not_commits { - let path = Path::from(not_commit); - assert!(!path.is_commit_file()); - } - } - #[tokio::test] async fn test_checkpoint_stream_parquet_read() { let metadata = Metadata { @@ -846,7 +823,7 @@ pub(super) mod tests { .await .unwrap(); - create_checkpoint_for(commit.version, &commit.snapshot, log_store.as_ref(), None) + create_checkpoint_for(commit.version as u64, log_store.as_ref(), None) .await .unwrap(); @@ -854,25 +831,21 @@ pub(super) mod tests { assert_eq!(commit.metrics.num_log_files_cleaned_up, 0); assert!(!commit.metrics.new_checkpoint_created); - let batches = LogSegment::try_new( - &Path::default(), - Some(commit.version), - log_store.object_store(None).as_ref(), - ) - .await - .unwrap() - .checkpoint_stream( - log_store.object_store(None), - &StructType::new(vec![ - ActionType::Metadata.schema_field().clone(), - ActionType::Protocol.schema_field().clone(), - ActionType::Add.schema_field().clone(), - ]), - &Default::default(), - ) - .try_collect::>() - .await - .unwrap(); + let batches = LogSegment::try_new(&log_store, Some(commit.version)) + .await + .unwrap() + .checkpoint_stream( + &log_store, + &StructType::new(vec![ + ActionType::Metadata.schema_field().clone(), + ActionType::Protocol.schema_field().clone(), + ActionType::Add.schema_field().clone(), + ]), + &Default::default(), + ) + .try_collect::>() + .await + .unwrap(); let batch = arrow::compute::concat_batches(&batches[0].schema(), batches.iter()).unwrap(); diff --git a/crates/core/src/kernel/snapshot/mod.rs b/crates/core/src/kernel/snapshot/mod.rs index 51bafb2c99..c754d702d1 100644 --- a/crates/core/src/kernel/snapshot/mod.rs +++ b/crates/core/src/kernel/snapshot/mod.rs @@ -17,17 +17,18 @@ use std::collections::{HashMap, HashSet}; use std::io::{BufRead, BufReader, Cursor}; -use std::sync::Arc; use ::serde::{Deserialize, Serialize}; use arrow_array::RecordBatch; +use delta_kernel::path::{LogPathFileType, ParsedLogPath}; use futures::stream::BoxStream; use futures::{StreamExt, TryStreamExt}; use object_store::path::Path; use object_store::ObjectStore; use tracing::warn; +use url::Url; -use self::log_segment::{LogSegment, PathExt}; +use self::log_segment::LogSegment; use self::parse::{read_adds, read_removes}; use self::replay::{LogMapper, LogReplayScanner, ReplayStream}; use self::visitors::*; @@ -36,9 +37,9 @@ use super::{ Transaction, }; use crate::kernel::parse::read_cdf_adds; +use crate::kernel::transaction::{CommitData, PROTOCOL}; use crate::kernel::{ActionType, StructType}; use crate::logstore::LogStore; -use crate::operations::transaction::CommitData; use crate::table::config::TableConfig; use crate::{DeltaResult, DeltaTableConfig, DeltaTableError}; @@ -54,26 +55,29 @@ pub mod visitors; /// A snapshot of a Delta table #[derive(Debug, Serialize, Deserialize, Clone, PartialEq)] pub struct Snapshot { + /// Log segment containing all log files in the snapshot log_segment: LogSegment, + /// Configuration for the current session config: DeltaTableConfig, + /// Protocol of the Delta table protocol: Protocol, + /// Metadata of the Delta table metadata: Metadata, + /// Logical table schema schema: StructType, - // TODO make this an URL - /// path of the table root within the object store - table_url: String, + /// Fully qualified URL of the table + table_url: Url, } impl Snapshot { /// Create a new [`Snapshot`] instance pub async fn try_new( - table_root: &Path, - store: Arc, + log_store: &dyn LogStore, config: DeltaTableConfig, version: Option, ) -> DeltaResult { - let log_segment = LogSegment::try_new(table_root, version, store.as_ref()).await?; - let (protocol, metadata) = log_segment.read_metadata(store.clone(), &config).await?; + let log_segment = LogSegment::try_new(log_store, version).await?; + let (protocol, metadata) = log_segment.read_metadata(log_store, &config).await?; if metadata.is_none() || protocol.is_none() { return Err(DeltaTableError::Generic( "Cannot read metadata from log segment".into(), @@ -81,22 +85,26 @@ impl Snapshot { }; let (metadata, protocol) = (metadata.unwrap(), protocol.unwrap()); let schema = serde_json::from_str(&metadata.schema_string)?; + + PROTOCOL.can_read_from_protocol(&protocol)?; + Ok(Self { log_segment, config, protocol, metadata, schema, - table_url: table_root.to_string(), + table_url: log_store.config().location.clone(), }) } #[cfg(test)] pub fn new_test<'a>( commits: impl IntoIterator, + table_root: &Path, ) -> DeltaResult<(Self, RecordBatch)> { use arrow_select::concat::concat_batches; - let (log_segment, batches) = LogSegment::new_test(commits)?; + let (log_segment, batches) = LogSegment::new_test(commits, table_root)?; let batch = batches.into_iter().collect::, _>>()?; let batch = concat_batches(&batch[0].schema(), &batch)?; let protocol = parse::read_protocol(&batch)?.unwrap(); @@ -109,7 +117,7 @@ impl Snapshot { protocol, metadata, schema, - table_url: Path::default().to_string(), + table_url: Url::parse("dummy:///").unwrap(), }, batch, )) @@ -118,7 +126,7 @@ impl Snapshot { /// Update the snapshot to the given version pub async fn update( &mut self, - log_store: Arc, + log_store: &dyn LogStore, target_version: Option, ) -> DeltaResult<()> { self.update_inner(log_store, target_version).await?; @@ -127,7 +135,7 @@ impl Snapshot { async fn update_inner( &mut self, - log_store: Arc, + log_store: &dyn LogStore, target_version: Option, ) -> DeltaResult> { if let Some(version) = target_version { @@ -138,20 +146,13 @@ impl Snapshot { return Err(DeltaTableError::Generic("Cannot downgrade snapshot".into())); } } - let log_segment = LogSegment::try_new_slice( - &Path::default(), - self.version() + 1, - target_version, - log_store.as_ref(), - ) - .await?; + let log_segment = + LogSegment::try_new_slice(self.version() + 1, target_version, log_store).await?; if log_segment.commit_files.is_empty() && log_segment.checkpoint_files.is_empty() { return Ok(None); } - let (protocol, metadata) = log_segment - .read_metadata(log_store.object_store(None).clone(), &self.config) - .await?; + let (protocol, metadata) = log_segment.read_metadata(log_store, &self.config).await?; if let Some(protocol) = protocol { self.protocol = protocol; } @@ -200,8 +201,8 @@ impl Snapshot { } /// Get the table root of the snapshot - pub fn table_root(&self) -> Path { - Path::from(self.table_url.clone()) + pub(crate) fn table_root_path(&self) -> DeltaResult { + Ok(Path::from_url_path(self.table_url.path())?) } /// Well known table configuration @@ -212,7 +213,7 @@ impl Snapshot { /// Get the files in the snapshot pub fn files<'a>( &self, - store: Arc, + log_store: &dyn LogStore, visitors: &'a mut Vec>, ) -> DeltaResult>>> { let mut schema_actions: HashSet<_> = @@ -220,14 +221,14 @@ impl Snapshot { schema_actions.insert(ActionType::Add); let checkpoint_stream = self.log_segment.checkpoint_stream( - store.clone(), + log_store, &StructType::new(schema_actions.iter().map(|a| a.schema_field().clone())), &self.config, ); schema_actions.insert(ActionType::Remove); let log_stream = self.log_segment.commit_stream( - store.clone(), + log_store, &StructType::new(schema_actions.iter().map(|a| a.schema_field().clone())), &self.config, )?; @@ -238,10 +239,12 @@ impl Snapshot { /// Get the commit infos in the snapshot pub(crate) async fn commit_infos( &self, - store: Arc, + log_store: &dyn LogStore, limit: Option, ) -> DeltaResult>>> { - let log_root = self.table_root().child("_delta_log"); + let store = log_store.root_object_store(None); + + let log_root = self.table_root_path()?.child("_delta_log"); let start_from = log_root.child( format!( "{:020}", @@ -252,14 +255,19 @@ impl Snapshot { .as_str(), ); + let dummy_url = url::Url::parse("memory:///").unwrap(); let mut commit_files = Vec::new(); for meta in store .list_with_offset(Some(&log_root), &start_from) .try_collect::>() .await? { - if meta.location.is_commit_file() { - commit_files.push(meta); + // safety: object store path are always valid urls paths. + let dummy_path = dummy_url.join(meta.location.as_ref()).unwrap(); + if let Some(parsed_path) = ParsedLogPath::try_from(dummy_path)? { + if matches!(parsed_path.file_type, LogPathFileType::Commit) { + commit_files.push(meta); + } } } commit_files.sort_unstable_by(|a, b| b.location.cmp(&a.location)); @@ -284,16 +292,18 @@ impl Snapshot { pub(crate) fn tombstones( &self, - store: Arc, + log_store: &dyn LogStore, ) -> DeltaResult>>> { let log_stream = self.log_segment.commit_stream( - store.clone(), + log_store, &log_segment::TOMBSTONE_SCHEMA, &self.config, )?; - let checkpoint_stream = - self.log_segment - .checkpoint_stream(store, &log_segment::TOMBSTONE_SCHEMA, &self.config); + let checkpoint_stream = self.log_segment.checkpoint_stream( + log_store, + &log_segment::TOMBSTONE_SCHEMA, + &self.config, + ); Ok(log_stream .chain(checkpoint_stream) @@ -330,7 +340,7 @@ pub struct EagerSnapshot { // additional actions that should be tracked during log replay. tracked_actions: HashSet, - transactions: Option>, + pub(crate) transactions: Option>, // NOTE: this is a Vec of RecordBatch instead of a single RecordBatch because // we do not yet enforce a consistent schema across all batches we read from the log. @@ -340,18 +350,16 @@ pub struct EagerSnapshot { impl EagerSnapshot { /// Create a new [`EagerSnapshot`] instance pub async fn try_new( - table_root: &Path, - store: Arc, + log_store: &dyn LogStore, config: DeltaTableConfig, version: Option, ) -> DeltaResult { - Self::try_new_with_visitor(table_root, store, config, version, Default::default()).await + Self::try_new_with_visitor(log_store, config, version, Default::default()).await } /// Create a new [`EagerSnapshot`] instance pub async fn try_new_with_visitor( - table_root: &Path, - store: Arc, + log_store: &dyn LogStore, config: DeltaTableConfig, version: Option, tracked_actions: HashSet, @@ -360,11 +368,15 @@ impl EagerSnapshot { .iter() .flat_map(get_visitor) .collect::>(); - let snapshot = - Snapshot::try_new(table_root, store.clone(), config.clone(), version).await?; + let snapshot = Snapshot::try_new(log_store, config.clone(), version).await?; let files = match config.require_files { - true => snapshot.files(store, &mut visitors)?.try_collect().await?, + true => { + snapshot + .files(log_store, &mut visitors)? + .try_collect() + .await? + } false => vec![], }; @@ -398,8 +410,11 @@ impl EagerSnapshot { } #[cfg(test)] - pub fn new_test<'a>(commits: impl IntoIterator) -> DeltaResult { - let (snapshot, batch) = Snapshot::new_test(commits)?; + pub fn new_test<'a>( + commits: impl IntoIterator, + table_root: &Path, + ) -> DeltaResult { + let (snapshot, batch) = Snapshot::new_test(commits, table_root)?; let mut files = Vec::new(); let mut scanner = LogReplayScanner::new(); files.push(scanner.process_files_batch(&batch, true)?); @@ -419,7 +434,7 @@ impl EagerSnapshot { /// Update the snapshot to the given version pub async fn update( &mut self, - log_store: Arc, + log_store: &dyn LogStore, target_version: Option, ) -> DeltaResult<()> { if Some(self.version()) == target_version { @@ -428,7 +443,7 @@ impl EagerSnapshot { let new_slice = self .snapshot - .update_inner(log_store.clone(), target_version) + .update_inner(log_store, target_version) .await?; if new_slice.is_none() { @@ -454,21 +469,13 @@ impl EagerSnapshot { let read_schema = StructType::new(schema_actions.iter().map(|a| a.schema_field().clone())); new_slice - .checkpoint_stream( - log_store.object_store(None), - &read_schema, - &self.snapshot.config, - ) + .checkpoint_stream(log_store, &read_schema, &self.snapshot.config) .boxed() }; schema_actions.insert(ActionType::Remove); let read_schema = StructType::new(schema_actions.iter().map(|a| a.schema_field().clone())); - let log_stream = new_slice.commit_stream( - log_store.object_store(None).clone(), - &read_schema, - &self.snapshot.config, - )?; + let log_stream = new_slice.commit_stream(log_store, &read_schema, &self.snapshot.config)?; let mapper = LogMapper::try_new(&self.snapshot, None)?; @@ -518,8 +525,8 @@ impl EagerSnapshot { } /// Get the table root of the snapshot - pub fn table_root(&self) -> Path { - self.snapshot.table_root() + pub(crate) fn table_root_path(&self) -> DeltaResult { + self.snapshot.table_root_path() } /// Get the table config which is loaded with of the snapshot @@ -558,14 +565,16 @@ impl EagerSnapshot { } /// Iterate over all latest app transactions - pub fn transactions(&self) -> DeltaResult + '_> { - self.transactions + pub async fn transaction_version(&self, app_id: impl AsRef) -> DeltaResult> { + Ok(self + .transactions .as_ref() - .map(|t| t.values().cloned()) .ok_or(DeltaTableError::Generic( "Transactions are not available. Please enable tracking of transactions." .to_string(), - )) + ))? + .get(app_id.as_ref()) + .map(|txn| txn.version)) } /// Advance the snapshot based on the given commit actions @@ -603,7 +612,7 @@ impl EagerSnapshot { let read_schema = StructType::new(schema_actions.iter().map(|a| a.schema_field().clone())); let actions = self.snapshot.log_segment.advance( send, - &self.table_root(), + &self.table_root_path()?, &read_schema, &self.snapshot.config, )?; @@ -813,49 +822,36 @@ fn find_nested_field<'a>( mod tests { use std::collections::HashMap; - use chrono::Utc; - use deltalake_test::utils::*; use futures::TryStreamExt; use itertools::Itertools; use super::log_segment::tests::{concurrent_checkpoint, test_log_segment}; use super::replay::tests::test_log_replay; use super::*; - use crate::kernel::Remove; use crate::protocol::{DeltaOperation, SaveMode}; - use crate::test_utils::ActionFactory; + use crate::test_utils::{assert_batches_sorted_eq, ActionFactory, TestResult, TestTables}; #[tokio::test] async fn test_snapshots() -> TestResult { - let context = IntegrationContext::new(Box::::default())?; - context.load_table(TestTables::Checkpoints).await?; - context.load_table(TestTables::Simple).await?; - context.load_table(TestTables::SimpleWithCheckpoint).await?; - context.load_table(TestTables::WithDvSmall).await?; - - test_log_segment(&context).await?; - test_log_replay(&context).await?; - test_snapshot(&context).await?; - test_eager_snapshot(&context).await?; + test_log_segment().await?; + test_log_replay().await?; + test_snapshot().await?; + test_eager_snapshot().await?; Ok(()) } + #[ignore] #[tokio::test(flavor = "multi_thread", worker_threads = 2)] async fn test_concurrent_checkpoint() -> TestResult { - let context = IntegrationContext::new(Box::::default())?; - concurrent_checkpoint(&context).await?; + concurrent_checkpoint().await?; Ok(()) } - async fn test_snapshot(context: &IntegrationContext) -> TestResult { - let store = context - .table_builder(TestTables::Simple) - .build_storage()? - .object_store(None); + async fn test_snapshot() -> TestResult { + let log_store = TestTables::Simple.table_builder().build_storage()?; - let snapshot = - Snapshot::try_new(&Path::default(), store.clone(), Default::default(), None).await?; + let snapshot = Snapshot::try_new(&log_store, Default::default(), None).await?; let bytes = serde_json::to_vec(&snapshot).unwrap(); let actual: Snapshot = serde_json::from_slice(&bytes).unwrap(); @@ -866,7 +862,7 @@ mod tests { assert_eq!(snapshot.schema(), &expected); let infos = snapshot - .commit_infos(store.clone(), None) + .commit_infos(&log_store, None) .await? .try_collect::>() .await?; @@ -874,14 +870,14 @@ mod tests { assert_eq!(infos.len(), 5); let tombstones = snapshot - .tombstones(store.clone())? + .tombstones(&log_store)? .try_collect::>() .await?; let tombstones = tombstones.into_iter().flatten().collect_vec(); assert_eq!(tombstones.len(), 31); let batches = snapshot - .files(store.clone(), &mut vec![])? + .files(&log_store, &mut vec![])? .try_collect::>() .await?; let expected = [ @@ -897,21 +893,12 @@ mod tests { ]; assert_batches_sorted_eq!(expected, &batches); - let store = context - .table_builder(TestTables::Checkpoints) - .build_storage()? - .object_store(None); + let log_store = TestTables::Checkpoints.table_builder().build_storage()?; for version in 0..=12 { - let snapshot = Snapshot::try_new( - &Path::default(), - store.clone(), - Default::default(), - Some(version), - ) - .await?; + let snapshot = Snapshot::try_new(&log_store, Default::default(), Some(version)).await?; let batches = snapshot - .files(store.clone(), &mut vec![])? + .files(&log_store, &mut vec![])? .try_collect::>() .await?; let num_files = batches.iter().map(|b| b.num_rows() as i64).sum::(); @@ -921,15 +908,10 @@ mod tests { Ok(()) } - async fn test_eager_snapshot(context: &IntegrationContext) -> TestResult { - let store = context - .table_builder(TestTables::Simple) - .build_storage()? - .object_store(None); + async fn test_eager_snapshot() -> TestResult { + let log_store = TestTables::Simple.table_builder().build_storage()?; - let snapshot = - EagerSnapshot::try_new(&Path::default(), store.clone(), Default::default(), None) - .await?; + let snapshot = EagerSnapshot::try_new(&log_store, Default::default(), None).await?; let bytes = serde_json::to_vec(&snapshot).unwrap(); let actual: EagerSnapshot = serde_json::from_slice(&bytes).unwrap(); @@ -939,19 +921,11 @@ mod tests { let expected: StructType = serde_json::from_str(schema_string)?; assert_eq!(snapshot.schema(), &expected); - let store = context - .table_builder(TestTables::Checkpoints) - .build_storage()? - .object_store(None); + let log_store = TestTables::Checkpoints.table_builder().build_storage()?; for version in 0..=12 { - let snapshot = EagerSnapshot::try_new( - &Path::default(), - store.clone(), - Default::default(), - Some(version), - ) - .await?; + let snapshot = + EagerSnapshot::try_new(&log_store, Default::default(), Some(version)).await?; let batches = snapshot.file_actions()?.collect::>(); assert_eq!(batches.len(), version as usize); } @@ -959,76 +933,6 @@ mod tests { Ok(()) } - #[tokio::test] - async fn test_eager_snapshot_advance() -> TestResult { - let context = IntegrationContext::new(Box::::default())?; - context.load_table(TestTables::Simple).await?; - - let store = context - .table_builder(TestTables::Simple) - .build_storage()? - .object_store(None); - - let mut snapshot = - EagerSnapshot::try_new(&Path::default(), store.clone(), Default::default(), None) - .await?; - - let version = snapshot.version(); - - let files = snapshot.file_actions()?.enumerate().collect_vec(); - let num_files = files.len(); - - let split = files.split(|(idx, _)| *idx == num_files / 2).collect_vec(); - assert!(split.len() == 2 && !split[0].is_empty() && !split[1].is_empty()); - let (first, second) = split.into_iter().next_tuple().unwrap(); - - let removes = first - .iter() - .map(|(_, add)| { - Remove { - path: add.path.clone(), - size: Some(add.size), - data_change: add.data_change, - deletion_timestamp: Some(Utc::now().timestamp_millis()), - extended_file_metadata: Some(true), - partition_values: Some(add.partition_values.clone()), - tags: add.tags.clone(), - deletion_vector: add.deletion_vector.clone(), - base_row_id: add.base_row_id, - default_row_commit_version: add.default_row_commit_version, - } - .into() - }) - .collect_vec(); - - let operation = DeltaOperation::Write { - mode: SaveMode::Append, - partition_by: None, - predicate: None, - }; - - let actions = vec![CommitData::new( - removes, - operation, - HashMap::new(), - Vec::new(), - )]; - - let new_version = snapshot.advance(&actions)?; - assert_eq!(new_version, version + 1); - - let new_files = snapshot.file_actions()?.map(|f| f.path).collect::>(); - assert_eq!(new_files.len(), num_files - first.len()); - assert!(first - .iter() - .all(|(_, add)| { !new_files.contains(&add.path) })); - assert!(second - .iter() - .all(|(_, add)| { new_files.contains(&add.path) })); - - Ok(()) - } - #[test] fn test_partition_schema() { let schema = StructType::new(vec![ @@ -1054,14 +958,15 @@ mod tests { HashMap::new(), vec![], ); - let (log_segment, _) = LogSegment::new_test(vec![&commit_data]).unwrap(); + + let (log_segment, _) = LogSegment::new_test(vec![&commit_data], &Path::default()).unwrap(); let snapshot = Snapshot { log_segment: log_segment.clone(), protocol: protocol.clone(), metadata, schema: schema.clone(), - table_url: "table".to_string(), + table_url: Url::parse("dummy:///").unwrap(), config: Default::default(), }; @@ -1082,7 +987,7 @@ mod tests { HashMap::new(), vec![], ); - let (log_segment, _) = LogSegment::new_test(vec![&commit_data]).unwrap(); + let (log_segment, _) = LogSegment::new_test(vec![&commit_data], &Path::default()).unwrap(); let snapshot = Snapshot { log_segment, @@ -1090,7 +995,7 @@ mod tests { protocol: protocol.clone(), metadata, schema: schema.clone(), - table_url: "table".to_string(), + table_url: Url::parse("dummy:///").unwrap(), }; assert_eq!(snapshot.partitions_schema(None).unwrap(), None); diff --git a/crates/core/src/kernel/snapshot/parse.rs b/crates/core/src/kernel/snapshot/parse.rs index 3bf0ff295a..c1e2f6f3ef 100644 --- a/crates/core/src/kernel/snapshot/parse.rs +++ b/crates/core/src/kernel/snapshot/parse.rs @@ -3,7 +3,7 @@ use arrow_array::{ Array, BooleanArray, Int32Array, Int64Array, ListArray, MapArray, StringArray, StructArray, }; -use delta_kernel::table_features::{ReaderFeatures, WriterFeatures}; +use delta_kernel::table_features::{ReaderFeature, WriterFeature}; use percent_encoding::percent_decode_str; use crate::kernel::arrow::extract::{self as ex, ProvidesColumnByName}; @@ -66,13 +66,13 @@ pub(super) fn read_protocol(batch: &dyn ProvidesColumnByName) -> DeltaResult::try_into(v.as_str())) + .map(|v| TryInto::::try_into(v.as_str())) .filter_map(|v| v.ok()) .collect() }), writer_features: collect_string_list(&maybe_writer_features, idx).map(|v| { v.into_iter() - .map(|v| TryInto::::try_into(v.as_str())) + .map(|v| TryInto::::try_into(v.as_str())) .filter_map(|v| v.ok()) .collect() }), @@ -167,7 +167,6 @@ pub(super) fn read_adds(array: &dyn ProvidesColumnByName) -> DeltaResult ReplayStream<'a, S> { snapshot: &Snapshot, visitors: &'a mut Vec>, ) -> DeltaResult { - let stats_schema = Arc::new((&snapshot.stats_schema(None)?).try_into()?); + let stats_schema = Arc::new((&snapshot.stats_schema(None)?).try_into_arrow()?); let partitions_schema = snapshot.partitions_schema(None)?.map(Arc::new); let mapper = Arc::new(LogMapper { stats_schema, @@ -82,7 +83,7 @@ impl LogMapper { table_schema: Option<&StructType>, ) -> DeltaResult { Ok(Self { - stats_schema: Arc::new((&snapshot.stats_schema(table_schema)?).try_into()?), + stats_schema: Arc::new((&snapshot.stats_schema(table_schema)?).try_into_arrow()?), partitions_schema: snapshot.partitions_schema(table_schema)?.map(Arc::new), config: snapshot.config.clone(), }) @@ -290,13 +291,13 @@ fn parse_partitions(batch: RecordBatch, partition_schema: &StructType) -> DeltaR _ => panic!("unexpected scalar type"), })), ) as ArrayRef, - PrimitiveType::Decimal(p, s) => Arc::new( + PrimitiveType::Decimal(decimal) => Arc::new( Decimal128Array::from_iter(values.iter().map(|v| match v { - Scalar::Decimal(d, _, _) => Some(*d), + Scalar::Decimal(decimal) => Some(decimal.bits()), Scalar::Null(_) => None, _ => panic!("unexpected scalar type"), })) - .with_precision_and_scale(*p, *s as i8)?, + .with_precision_and_scale(decimal.precision(), decimal.scale() as i8)?, ) as ArrayRef, }; Ok(arr) @@ -314,7 +315,7 @@ fn parse_partitions(batch: RecordBatch, partition_schema: &StructType) -> DeltaR Fields::from( partition_schema .fields() - .map(|f| f.try_into()) + .map(|f| f.try_into_arrow()) .collect::, _>>()?, ), columns, @@ -602,35 +603,34 @@ pub(super) mod tests { use std::sync::Arc; use arrow_select::concat::concat_batches; + use delta_kernel::engine::arrow_conversion::TryIntoKernel; use delta_kernel::schema::DataType; - use deltalake_test::utils::*; use futures::TryStreamExt; use object_store::path::Path; use super::super::{log_segment::LogSegment, partitions_schema, stats_schema}; use super::*; + use crate::kernel::transaction::CommitData; use crate::kernel::{models::ActionType, StructType}; - use crate::operations::transaction::CommitData; use crate::protocol::DeltaOperation; use crate::table::config::TableConfig; - use crate::test_utils::{ActionFactory, TestResult, TestSchemas}; + use crate::test_utils::{ActionFactory, TestResult, TestSchemas, TestTables}; - pub(crate) async fn test_log_replay(context: &IntegrationContext) -> TestResult { + pub(crate) async fn test_log_replay() -> TestResult { let log_schema = Arc::new(StructType::new(vec![ ActionType::Add.schema_field().clone(), ActionType::Remove.schema_field().clone(), ])); - let store = context - .table_builder(TestTables::SimpleWithCheckpoint) - .build_storage()? - .object_store(None); + let log_store = TestTables::SimpleWithCheckpoint + .table_builder() + .build_storage()?; - let segment = LogSegment::try_new(&Path::default(), Some(9), store.as_ref()).await?; + let segment = LogSegment::try_new(&log_store, Some(9)).await?; let mut scanner = LogReplayScanner::new(); let batches = segment - .commit_stream(store.clone(), &log_schema, &Default::default())? + .commit_stream(&log_store, &log_schema, &Default::default())? .try_collect::>() .await?; let batch = concat_batches(&batches[0].schema(), &batches)?; @@ -643,13 +643,10 @@ pub(super) mod tests { let filtered = scanner.process_files_batch(&batch, true)?; assert_eq!(filtered.schema().fields().len(), 1); - let store = context - .table_builder(TestTables::Simple) - .build_storage()? - .object_store(None); - let segment = LogSegment::try_new(&Path::default(), None, store.as_ref()).await?; + let log_store = TestTables::Simple.table_builder().build_storage()?; + let segment = LogSegment::try_new(&log_store, None).await?; let batches = segment - .commit_stream(store.clone(), &log_schema, &Default::default())? + .commit_stream(&log_store, &log_schema, &Default::default())? .try_collect::>() .await?; @@ -686,7 +683,7 @@ pub(super) mod tests { app_metadata: Default::default(), app_transactions: Default::default(), }; - let (_, maybe_batches) = LogSegment::new_test(&[commit_data])?; + let (_, maybe_batches) = LogSegment::new_test(&[commit_data], &Path::default())?; let batches = maybe_batches.into_iter().collect::, _>>()?; let batch = concat_batches(&batches[0].schema(), &batches)?; @@ -695,11 +692,11 @@ pub(super) mod tests { assert!(ex::extract_and_cast_opt::(&batch, "add.stats_parsed").is_none()); let stats_schema = stats_schema(schema, table_config)?; - let new_batch = parse_stats(batch, Arc::new((&stats_schema).try_into()?), &config)?; + let new_batch = parse_stats(batch, Arc::new((&stats_schema).try_into_arrow()?), &config)?; assert!(ex::extract_and_cast_opt::(&new_batch, "add.stats_parsed").is_some()); let parsed_col = ex::extract_and_cast::(&new_batch, "add.stats_parsed")?; - let delta_type: DataType = parsed_col.data_type().try_into()?; + let delta_type: DataType = parsed_col.data_type().try_into_kernel()?; match delta_type { DataType::Struct(fields) => { @@ -749,7 +746,7 @@ pub(super) mod tests { app_metadata: Default::default(), app_transactions: Default::default(), }; - let (_, maybe_batches) = LogSegment::new_test(&[commit_data])?; + let (_, maybe_batches) = LogSegment::new_test(&[commit_data], &Path::default())?; let batches = maybe_batches.into_iter().collect::, _>>()?; let batch = concat_batches(&batches[0].schema(), &batches)?; @@ -768,7 +765,7 @@ pub(super) mod tests { ); let parsed_col = ex::extract_and_cast::(&new_batch, "add.partitionValues_parsed")?; - let delta_type: DataType = parsed_col.data_type().try_into()?; + let delta_type: DataType = parsed_col.data_type().try_into_kernel()?; match delta_type { DataType::Struct(fields) => { diff --git a/crates/core/src/kernel/snapshot/serde.rs b/crates/core/src/kernel/snapshot/serde.rs index 9ffbc2deaa..84a0f135fb 100644 --- a/crates/core/src/kernel/snapshot/serde.rs +++ b/crates/core/src/kernel/snapshot/serde.rs @@ -29,7 +29,7 @@ impl Serialize for LogSegment { .iter() .map(|f| FileInfo { path: f.location.to_string(), - size: f.size, + size: f.size as usize, last_modified: f.last_modified.timestamp_nanos_opt().unwrap(), e_tag: f.e_tag.clone(), version: f.version.clone(), @@ -40,7 +40,7 @@ impl Serialize for LogSegment { .iter() .map(|f| FileInfo { path: f.location.to_string(), - size: f.size, + size: f.size as usize, last_modified: f.last_modified.timestamp_nanos_opt().unwrap(), e_tag: f.e_tag.clone(), version: f.version.clone(), @@ -88,7 +88,7 @@ impl<'de> Visitor<'de> for LogSegmentVisitor { let nano_seconds = (f.last_modified % 1_000_000_000) as u32; ObjectMeta { location: f.path.into(), - size: f.size, + size: f.size as u64, last_modified: Utc.timestamp_opt(seconds, nano_seconds).single().unwrap(), version: f.version, e_tag: f.e_tag, @@ -99,7 +99,7 @@ impl<'de> Visitor<'de> for LogSegmentVisitor { .into_iter() .map(|f| ObjectMeta { location: f.path.into(), - size: f.size, + size: f.size as u64, last_modified: DateTime::from_timestamp_millis(f.last_modified).unwrap(), version: None, diff --git a/crates/core/src/operations/transaction/application.rs b/crates/core/src/kernel/transaction/application.rs similarity index 65% rename from crates/core/src/operations/transaction/application.rs rename to crates/core/src/kernel/transaction/application.rs index c2f08e8838..e42535793b 100644 --- a/crates/core/src/operations/transaction/application.rs +++ b/crates/core/src/kernel/transaction/application.rs @@ -1,10 +1,11 @@ #[cfg(test)] mod tests { use crate::{ - checkpoints, kernel::Transaction, operations::transaction::CommitProperties, + checkpoints, kernel::transaction::CommitProperties, kernel::Transaction, protocol::SaveMode, writer::test_utils::get_record_batch, DeltaOps, DeltaTableBuilder, }; + #[cfg(feature = "datafusion")] #[tokio::test] async fn test_app_txn_workload() { // Test that the transaction ids can be read from different scenarios @@ -29,13 +30,22 @@ mod tests { ) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_files_count(), 2); let app_txns = table.get_app_transaction_version(); assert_eq!(app_txns.len(), 1); assert_eq!(app_txns.get("my-app").map(|t| t.version), Some(1)); + let log_store = table.log_store(); + let txn_version = table + .snapshot() + .unwrap() + .transaction_version(log_store.as_ref(), "my-app") + .await + .unwrap(); + assert_eq!(txn_version, Some(1)); + // Test Txn Id can be read from existing table let mut table2 = DeltaTableBuilder::from_uri(tmp_path.to_str().unwrap()) @@ -46,6 +56,13 @@ mod tests { assert_eq!(app_txns2.len(), 1); assert_eq!(app_txns2.get("my-app").map(|t| t.version), Some(1)); + let txn_version = table2 + .snapshot() + .unwrap() + .transaction_version(log_store.as_ref(), "my-app") + .await + .unwrap(); + assert_eq!(txn_version, Some(1)); // Write new data to the table and check that `update` functions work @@ -58,16 +75,30 @@ mod tests { .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let app_txns = table.get_app_transaction_version(); assert_eq!(app_txns.len(), 1); assert_eq!(app_txns.get("my-app").map(|t| t.version), Some(3)); + let txn_version = table + .snapshot() + .unwrap() + .transaction_version(log_store.as_ref(), "my-app") + .await + .unwrap(); + assert_eq!(txn_version, Some(3)); table2.update_incremental(None).await.unwrap(); - assert_eq!(table2.version(), 1); + assert_eq!(table2.version(), Some(1)); let app_txns2 = table2.get_app_transaction_version(); assert_eq!(app_txns2.len(), 1); assert_eq!(app_txns2.get("my-app").map(|t| t.version), Some(3)); + let txn_version = table2 + .snapshot() + .unwrap() + .transaction_version(log_store.as_ref(), "my-app") + .await + .unwrap(); + assert_eq!(txn_version, Some(3)); // Create a checkpoint and then load checkpoints::create_checkpoint(&table, None).await.unwrap(); @@ -75,9 +106,16 @@ mod tests { .load() .await .unwrap(); - let app_txns3 = table2.get_app_transaction_version(); + let app_txns3 = table3.get_app_transaction_version(); assert_eq!(app_txns3.len(), 1); assert_eq!(app_txns3.get("my-app").map(|t| t.version), Some(3)); - assert_eq!(table3.version(), 1); + assert_eq!(table3.version(), Some(1)); + let txn_version = table3 + .snapshot() + .unwrap() + .transaction_version(log_store.as_ref(), "my-app") + .await + .unwrap(); + assert_eq!(txn_version, Some(3)); } } diff --git a/crates/core/src/operations/transaction/conflict_checker.rs b/crates/core/src/kernel/transaction/conflict_checker.rs similarity index 99% rename from crates/core/src/operations/transaction/conflict_checker.rs rename to crates/core/src/kernel/transaction/conflict_checker.rs index 4f9824f56a..5d0d520277 100644 --- a/crates/core/src/operations/transaction/conflict_checker.rs +++ b/crates/core/src/kernel/transaction/conflict_checker.rs @@ -698,10 +698,12 @@ mod tests { actions: Vec, read_whole_table: bool, ) -> Result<(), CommitConflictError> { + use object_store::path::Path; + use crate::table::state::DeltaTableState; let setup_actions = setup.unwrap_or_else(init_table_actions); - let state = DeltaTableState::from_actions(setup_actions).unwrap(); + let state = DeltaTableState::from_actions(setup_actions, &Path::default()).unwrap(); let snapshot = state.snapshot(); let transaction_info = TransactionInfo::new(snapshot, reads, &actions, read_whole_table); let summary = WinningCommitSummary { diff --git a/crates/core/src/operations/transaction/mod.rs b/crates/core/src/kernel/transaction/mod.rs similarity index 96% rename from crates/core/src/operations/transaction/mod.rs rename to crates/core/src/kernel/transaction/mod.rs index 031d8e4986..a73d24c113 100644 --- a/crates/core/src/operations/transaction/mod.rs +++ b/crates/core/src/kernel/transaction/mod.rs @@ -86,22 +86,23 @@ use serde_json::Value; use tracing::*; use uuid::Uuid; -pub use self::conflict_checker::CommitConflictError; +use delta_kernel::table_features::{ReaderFeature, WriterFeature}; +use serde::{Deserialize, Serialize}; + use self::conflict_checker::{TransactionInfo, WinningCommitSummary}; -pub use self::protocol::INSTANCE as PROTOCOL; -use crate::checkpoints::{cleanup_expired_logs_for, create_checkpoint_for}; use crate::errors::DeltaTableError; use crate::kernel::{Action, CommitInfo, EagerSnapshot, Metadata, Protocol, Transaction}; +use crate::logstore::ObjectStoreRef; use crate::logstore::{CommitOrBytes, LogStoreRef}; +use crate::operations::CustomExecuteHandler; use crate::protocol::DeltaOperation; -use crate::storage::ObjectStoreRef; +use crate::protocol::{cleanup_expired_logs_for, create_checkpoint_for}; use crate::table::config::TableConfig; use crate::table::state::DeltaTableState; use crate::{crate_version, DeltaResult}; -use delta_kernel::table_features::{ReaderFeatures, WriterFeatures}; -use serde::{Deserialize, Serialize}; -use super::CustomExecuteHandler; +pub use self::conflict_checker::CommitConflictError; +pub use self::protocol::INSTANCE as PROTOCOL; #[cfg(test)] pub(crate) mod application; @@ -181,19 +182,19 @@ pub enum TransactionError { /// Error returned when unsupported reader features are required #[error("Unsupported reader features required: {0:?}")] - UnsupportedReaderFeatures(Vec), + UnsupportedReaderFeatures(Vec), /// Error returned when unsupported writer features are required #[error("Unsupported writer features required: {0:?}")] - UnsupportedWriterFeatures(Vec), + UnsupportedWriterFeatures(Vec), /// Error returned when writer features are required but not specified #[error("Writer features must be specified for writerversion >= 7, please specify: {0:?}")] - WriterFeaturesRequired(WriterFeatures), + WriterFeaturesRequired(WriterFeature), /// Error returned when reader features are required but not specified #[error("Reader features must be specified for reader version >= 3, please specify: {0:?}")] - ReaderFeaturesRequired(ReaderFeatures), + ReaderFeaturesRequired(ReaderFeature), /// The transaction failed to commit due to an error in an implementation-specific layer. /// Currently used by DynamoDb-backed S3 log store when database operations fail. @@ -683,9 +684,9 @@ impl<'a> std::future::IntoFuture for PreparedCommit<'a> { } steps -= 1; } - // Update snapshot to latest version after succesful conflict check + // Update snapshot to latest version after successful conflict check read_snapshot - .update(this.log_store.clone(), Some(latest_version)) + .update(&this.log_store, Some(latest_version)) .await?; } let version: i64 = latest_version + 1; @@ -759,7 +760,7 @@ impl PostCommit { // This may only occur during concurrent write actions. We need to update the state first to - 1 // then we can advance. snapshot - .update(self.log_store.clone(), Some(self.version - 1)) + .update(&self.log_store, Some(self.version - 1)) .await?; snapshot.advance(vec![&self.data])?; } else { @@ -810,8 +811,7 @@ impl PostCommit { .await? as u64; if num_log_files_cleaned_up > 0 { state = DeltaTableState::try_new( - &state.snapshot().table_root(), - self.log_store.object_store(None), + &self.log_store, state.load_config().clone(), Some(self.version), ) @@ -837,13 +837,9 @@ impl PostCommit { }, )) } else { - let state = DeltaTableState::try_new( - &Path::default(), - self.log_store.object_store(None), - Default::default(), - Some(self.version), - ) - .await?; + let state = + DeltaTableState::try_new(&self.log_store, Default::default(), Some(self.version)) + .await?; Ok(( state, PostCommitMetrics { @@ -867,8 +863,7 @@ impl PostCommit { let checkpoint_interval = table_state.config().checkpoint_interval() as i64; if ((version + 1) % checkpoint_interval) == 0 { - create_checkpoint_for(version, table_state, log_store.as_ref(), Some(operation_id)) - .await?; + create_checkpoint_for(version as u64, log_store.as_ref(), Some(operation_id)).await?; Ok(true) } else { Ok(false) @@ -925,13 +920,10 @@ impl std::future::IntoFuture for PostCommit { #[cfg(test)] mod tests { - use std::{collections::HashMap, sync::Arc}; + use std::sync::Arc; use super::*; - use crate::{ - logstore::{default_logstore::DefaultLogStore, LogStore}, - storage::commit_uri_from_version, - }; + use crate::logstore::{commit_uri_from_version, default_logstore::DefaultLogStore, LogStore}; use object_store::{memory::InMemory, ObjectStore, PutPayload}; use url::Url; @@ -948,10 +940,11 @@ mod tests { let store = Arc::new(InMemory::new()); let url = Url::parse("mem://what/is/this").unwrap(); let log_store = DefaultLogStore::new( + store.clone(), store.clone(), crate::logstore::LogStoreConfig { location: url, - options: HashMap::new().into(), + options: Default::default(), }, ); let version_path = Path::from("_delta_log/00000000000000000000.json"); diff --git a/crates/core/src/operations/transaction/protocol.rs b/crates/core/src/kernel/transaction/protocol.rs similarity index 79% rename from crates/core/src/operations/transaction/protocol.rs rename to crates/core/src/kernel/transaction/protocol.rs index c47578d6a1..4f3425c094 100644 --- a/crates/core/src/operations/transaction/protocol.rs +++ b/crates/core/src/kernel/transaction/protocol.rs @@ -1,64 +1,71 @@ use std::collections::HashSet; use std::sync::LazyLock; +use delta_kernel::table_features::{ReaderFeature, WriterFeature}; + use super::{TableReference, TransactionError}; -use crate::kernel::{contains_timestampntz, Action, EagerSnapshot, Schema}; +use crate::kernel::{contains_timestampntz, Action, EagerSnapshot, Protocol, Schema}; use crate::protocol::DeltaOperation; use crate::table::state::DeltaTableState; -use delta_kernel::table_features::{ReaderFeatures, WriterFeatures}; -static READER_V2: LazyLock> = - LazyLock::new(|| HashSet::from_iter([ReaderFeatures::ColumnMapping])); -static WRITER_V2: LazyLock> = - LazyLock::new(|| HashSet::from_iter([WriterFeatures::AppendOnly, WriterFeatures::Invariants])); -static WRITER_V3: LazyLock> = LazyLock::new(|| { +static READER_V2: LazyLock> = + LazyLock::new(|| HashSet::from_iter([ReaderFeature::ColumnMapping])); +#[cfg(feature = "datafusion")] +static WRITER_V2: LazyLock> = + LazyLock::new(|| HashSet::from_iter([WriterFeature::AppendOnly, WriterFeature::Invariants])); +// Invariants cannot work in the default builds where datafusion is not present currently, this +// feature configuration ensures that the writer doesn't pretend otherwise +#[cfg(not(feature = "datafusion"))] +static WRITER_V2: LazyLock> = + LazyLock::new(|| HashSet::from_iter([WriterFeature::AppendOnly])); +static WRITER_V3: LazyLock> = LazyLock::new(|| { HashSet::from_iter([ - WriterFeatures::AppendOnly, - WriterFeatures::Invariants, - WriterFeatures::CheckConstraints, + WriterFeature::AppendOnly, + WriterFeature::Invariants, + WriterFeature::CheckConstraints, ]) }); -static WRITER_V4: LazyLock> = LazyLock::new(|| { +static WRITER_V4: LazyLock> = LazyLock::new(|| { HashSet::from_iter([ - WriterFeatures::AppendOnly, - WriterFeatures::Invariants, - WriterFeatures::CheckConstraints, - WriterFeatures::ChangeDataFeed, - WriterFeatures::GeneratedColumns, + WriterFeature::AppendOnly, + WriterFeature::Invariants, + WriterFeature::CheckConstraints, + WriterFeature::ChangeDataFeed, + WriterFeature::GeneratedColumns, ]) }); -static WRITER_V5: LazyLock> = LazyLock::new(|| { +static WRITER_V5: LazyLock> = LazyLock::new(|| { HashSet::from_iter([ - WriterFeatures::AppendOnly, - WriterFeatures::Invariants, - WriterFeatures::CheckConstraints, - WriterFeatures::ChangeDataFeed, - WriterFeatures::GeneratedColumns, - WriterFeatures::ColumnMapping, + WriterFeature::AppendOnly, + WriterFeature::Invariants, + WriterFeature::CheckConstraints, + WriterFeature::ChangeDataFeed, + WriterFeature::GeneratedColumns, + WriterFeature::ColumnMapping, ]) }); -static WRITER_V6: LazyLock> = LazyLock::new(|| { +static WRITER_V6: LazyLock> = LazyLock::new(|| { HashSet::from_iter([ - WriterFeatures::AppendOnly, - WriterFeatures::Invariants, - WriterFeatures::CheckConstraints, - WriterFeatures::ChangeDataFeed, - WriterFeatures::GeneratedColumns, - WriterFeatures::ColumnMapping, - WriterFeatures::IdentityColumns, + WriterFeature::AppendOnly, + WriterFeature::Invariants, + WriterFeature::CheckConstraints, + WriterFeature::ChangeDataFeed, + WriterFeature::GeneratedColumns, + WriterFeature::ColumnMapping, + WriterFeature::IdentityColumns, ]) }); pub struct ProtocolChecker { - reader_features: HashSet, - writer_features: HashSet, + reader_features: HashSet, + writer_features: HashSet, } impl ProtocolChecker { /// Create a new protocol checker. pub fn new( - reader_features: HashSet, - writer_features: HashSet, + reader_features: HashSet, + writer_features: HashSet, ) -> Self { Self { reader_features, @@ -89,23 +96,23 @@ impl ProtocolChecker { schema: &Schema, ) -> Result<(), TransactionError> { let contains_timestampntz = contains_timestampntz(schema.fields()); - let required_features: Option<&HashSet> = + let required_features: Option<&HashSet> = match snapshot.protocol().min_writer_version { 0..=6 => None, _ => snapshot.protocol().writer_features.as_ref(), }; if let Some(table_features) = required_features { - if !table_features.contains(&WriterFeatures::TimestampWithoutTimezone) + if !table_features.contains(&WriterFeature::TimestampWithoutTimezone) && contains_timestampntz { return Err(TransactionError::WriterFeaturesRequired( - WriterFeatures::TimestampWithoutTimezone, + WriterFeature::TimestampWithoutTimezone, )); } } else if contains_timestampntz { return Err(TransactionError::WriterFeaturesRequired( - WriterFeatures::TimestampWithoutTimezone, + WriterFeature::TimestampWithoutTimezone, )); } Ok(()) @@ -113,12 +120,15 @@ impl ProtocolChecker { /// Check if delta-rs can read form the given delta table. pub fn can_read_from(&self, snapshot: &dyn TableReference) -> Result<(), TransactionError> { - let required_features: Option<&HashSet> = - match snapshot.protocol().min_reader_version { - 0 | 1 => None, - 2 => Some(&READER_V2), - _ => snapshot.protocol().reader_features.as_ref(), - }; + self.can_read_from_protocol(snapshot.protocol()) + } + + pub fn can_read_from_protocol(&self, protocol: &Protocol) -> Result<(), TransactionError> { + let required_features: Option<&HashSet> = match protocol.min_reader_version { + 0 | 1 => None, + 2 => Some(&READER_V2), + _ => protocol.reader_features.as_ref(), + }; if let Some(features) = required_features { let mut diff = features.difference(&self.reader_features).peekable(); if diff.peek().is_some() { @@ -136,7 +146,7 @@ impl ProtocolChecker { self.can_read_from(snapshot)?; let min_writer_version = snapshot.protocol().min_writer_version; - let required_features: Option<&HashSet> = match min_writer_version { + let required_features: Option<&HashSet> = match min_writer_version { 0 | 1 => None, 2 => Some(&WRITER_V2), 3 => Some(&WRITER_V3), @@ -176,9 +186,9 @@ impl ProtocolChecker { .writer_features .as_ref() .ok_or(TransactionError::WriterFeaturesRequired( - WriterFeatures::AppendOnly, + WriterFeature::AppendOnly, ))? - .contains(&WriterFeatures::AppendOnly) + .contains(&WriterFeature::AppendOnly) && snapshot.config().append_only() }; if append_only_enabled { @@ -208,24 +218,21 @@ impl ProtocolChecker { /// resulting version support is determined by the supported table feature set. pub static INSTANCE: LazyLock = LazyLock::new(|| { let mut reader_features = HashSet::new(); - reader_features.insert(ReaderFeatures::TimestampWithoutTimezone); - // reader_features.insert(ReaderFeatures::ColumnMapping); + reader_features.insert(ReaderFeature::TimestampWithoutTimezone); + // reader_features.insert(ReaderFeature::ColumnMapping); let mut writer_features = HashSet::new(); - writer_features.insert(WriterFeatures::AppendOnly); - writer_features.insert(WriterFeatures::TimestampWithoutTimezone); - #[cfg(feature = "cdf")] - { - writer_features.insert(WriterFeatures::ChangeDataFeed); - } + writer_features.insert(WriterFeature::AppendOnly); + writer_features.insert(WriterFeature::TimestampWithoutTimezone); #[cfg(feature = "datafusion")] { - writer_features.insert(WriterFeatures::Invariants); - writer_features.insert(WriterFeatures::CheckConstraints); - writer_features.insert(WriterFeatures::GeneratedColumns); + writer_features.insert(WriterFeature::ChangeDataFeed); + writer_features.insert(WriterFeature::Invariants); + writer_features.insert(WriterFeature::CheckConstraints); + writer_features.insert(WriterFeature::GeneratedColumns); } - // writer_features.insert(WriterFeatures::ColumnMapping); - // writer_features.insert(WriterFeatures::IdentityColumns); + // writer_features.insert(WriterFeature::ColumnMapping); + // writer_features.insert(WriterFeature::IdentityColumns); ProtocolChecker::new(reader_features, writer_features) }); @@ -234,6 +241,8 @@ pub static INSTANCE: LazyLock = LazyLock::new(|| { mod tests { use std::collections::HashMap; + use object_store::path::Path; + use super::*; use crate::kernel::DataType as DeltaDataType; use crate::kernel::{Action, Add, Metadata, PrimitiveType, Protocol, Remove}; @@ -287,7 +296,7 @@ mod tests { ]; let neutral_op = DeltaOperation::Update { predicate: None }; - let create_actions = |writer: i32, append: &str, feat: Vec| { + let create_actions = |writer: i32, append: &str, feat: Vec| { vec![ Action::Protocol(Protocol { min_reader_version: 1, @@ -306,7 +315,7 @@ mod tests { let checker = ProtocolChecker::new(HashSet::new(), WRITER_V2.clone()); let actions = create_actions(1, "true", vec![]); - let snapshot = DeltaTableState::from_actions(actions).unwrap(); + let snapshot = DeltaTableState::from_actions(actions, &Path::default()).unwrap(); let eager = snapshot.snapshot(); assert!(checker .can_commit(eager, &append_actions, &append_op) @@ -319,7 +328,7 @@ mod tests { .is_ok()); let actions = create_actions(2, "true", vec![]); - let snapshot = DeltaTableState::from_actions(actions).unwrap(); + let snapshot = DeltaTableState::from_actions(actions, &Path::default()).unwrap(); let eager = snapshot.snapshot(); assert!(checker .can_commit(eager, &append_actions, &append_op) @@ -332,7 +341,7 @@ mod tests { .is_ok()); let actions = create_actions(2, "false", vec![]); - let snapshot = DeltaTableState::from_actions(actions).unwrap(); + let snapshot = DeltaTableState::from_actions(actions, &Path::default()).unwrap(); let eager = snapshot.snapshot(); assert!(checker .can_commit(eager, &append_actions, &append_op) @@ -344,8 +353,8 @@ mod tests { .can_commit(eager, &neutral_actions, &neutral_op) .is_ok()); - let actions = create_actions(7, "true", vec![WriterFeatures::AppendOnly]); - let snapshot = DeltaTableState::from_actions(actions).unwrap(); + let actions = create_actions(7, "true", vec![WriterFeature::AppendOnly]); + let snapshot = DeltaTableState::from_actions(actions, &Path::default()).unwrap(); let eager = snapshot.snapshot(); assert!(checker .can_commit(eager, &append_actions, &append_op) @@ -357,8 +366,8 @@ mod tests { .can_commit(eager, &neutral_actions, &neutral_op) .is_ok()); - let actions = create_actions(7, "false", vec![WriterFeatures::AppendOnly]); - let snapshot = DeltaTableState::from_actions(actions).unwrap(); + let actions = create_actions(7, "false", vec![WriterFeature::AppendOnly]); + let snapshot = DeltaTableState::from_actions(actions, &Path::default()).unwrap(); let eager = snapshot.snapshot(); assert!(checker .can_commit(eager, &append_actions, &append_op) @@ -371,7 +380,7 @@ mod tests { .is_ok()); let actions = create_actions(7, "true", vec![]); - let snapshot = DeltaTableState::from_actions(actions).unwrap(); + let snapshot = DeltaTableState::from_actions(actions, &Path::default()).unwrap(); let eager = snapshot.snapshot(); assert!(checker .can_commit(eager, &append_actions, &append_op) @@ -395,7 +404,7 @@ mod tests { }), metadata_action(None).into(), ]; - let snapshot_1 = DeltaTableState::from_actions(actions).unwrap(); + let snapshot_1 = DeltaTableState::from_actions(actions, &Path::default()).unwrap(); let eager_1 = snapshot_1.snapshot(); assert!(checker_1.can_read_from(eager_1).is_ok()); assert!(checker_1.can_write_to(eager_1).is_ok()); @@ -409,7 +418,7 @@ mod tests { }), metadata_action(None).into(), ]; - let snapshot_2 = DeltaTableState::from_actions(actions).unwrap(); + let snapshot_2 = DeltaTableState::from_actions(actions, &Path::default()).unwrap(); let eager_2 = snapshot_2.snapshot(); assert!(checker_1.can_read_from(eager_2).is_err()); assert!(checker_1.can_write_to(eager_2).is_err()); @@ -426,7 +435,7 @@ mod tests { }), metadata_action(None).into(), ]; - let snapshot_3 = DeltaTableState::from_actions(actions).unwrap(); + let snapshot_3 = DeltaTableState::from_actions(actions, &Path::default()).unwrap(); let eager_3 = snapshot_3.snapshot(); assert!(checker_1.can_read_from(eager_3).is_err()); assert!(checker_1.can_write_to(eager_3).is_err()); @@ -446,7 +455,7 @@ mod tests { }), metadata_action(None).into(), ]; - let snapshot_4 = DeltaTableState::from_actions(actions).unwrap(); + let snapshot_4 = DeltaTableState::from_actions(actions, &Path::default()).unwrap(); let eager_4 = snapshot_4.snapshot(); assert!(checker_1.can_read_from(eager_4).is_err()); assert!(checker_1.can_write_to(eager_4).is_err()); @@ -469,7 +478,7 @@ mod tests { }), metadata_action(None).into(), ]; - let snapshot_5 = DeltaTableState::from_actions(actions).unwrap(); + let snapshot_5 = DeltaTableState::from_actions(actions, &Path::default()).unwrap(); let eager_5 = snapshot_5.snapshot(); assert!(checker_1.can_read_from(eager_5).is_err()); assert!(checker_1.can_write_to(eager_5).is_err()); @@ -495,7 +504,7 @@ mod tests { }), metadata_action(None).into(), ]; - let snapshot_6 = DeltaTableState::from_actions(actions).unwrap(); + let snapshot_6 = DeltaTableState::from_actions(actions, &Path::default()).unwrap(); let eager_6 = snapshot_6.snapshot(); assert!(checker_1.can_read_from(eager_6).is_err()); assert!(checker_1.can_write_to(eager_6).is_err()); @@ -524,7 +533,7 @@ mod tests { }), metadata_action(None).into(), ]; - let snapshot_7 = DeltaTableState::from_actions(actions).unwrap(); + let snapshot_7 = DeltaTableState::from_actions(actions, &Path::default()).unwrap(); let eager_7 = snapshot_7.snapshot(); assert!(checker_1.can_read_from(eager_7).is_err()); assert!(checker_1.can_write_to(eager_7).is_err()); @@ -553,11 +562,11 @@ mod tests { let checker_5 = ProtocolChecker::new(READER_V2.clone(), WRITER_V4.clone()); let actions = vec![ Action::Protocol( - Protocol::new(2, 4).append_writer_features(vec![WriterFeatures::ChangeDataFeed]), + Protocol::new(2, 4).append_writer_features(vec![WriterFeature::ChangeDataFeed]), ), metadata_action(None).into(), ]; - let snapshot_5 = DeltaTableState::from_actions(actions).unwrap(); + let snapshot_5 = DeltaTableState::from_actions(actions, &Path::default()).unwrap(); let eager_5 = snapshot_5.snapshot(); assert!(checker_5.can_write_to(eager_5).is_ok()); } @@ -570,19 +579,19 @@ mod tests { let checker_5 = ProtocolChecker::new(READER_V2.clone(), WRITER_V4.clone()); let actions = vec![ Action::Protocol( - Protocol::new(2, 4).append_writer_features([WriterFeatures::GeneratedColumns]), + Protocol::new(2, 4).append_writer_features([WriterFeature::GeneratedColumns]), ), metadata_action(None).into(), ]; - let snapshot_5 = DeltaTableState::from_actions(actions).unwrap(); + let snapshot_5 = DeltaTableState::from_actions(actions, &Path::default()).unwrap(); let eager_5 = snapshot_5.snapshot(); assert!(checker_5.can_write_to(eager_5).is_ok()); } #[tokio::test] async fn test_minwriter_v4_with_generated_columns_and_expressions() { - let checker_5 = ProtocolChecker::new(READER_V2.clone(), WRITER_V4.clone()); - let actions = vec![Action::Protocol(Protocol::new(2, 4))]; + let checker_5 = ProtocolChecker::new(Default::default(), WRITER_V4.clone()); + let actions = vec![Action::Protocol(Protocol::new(1, 4))]; let table: crate::DeltaTable = crate::DeltaOps::new_in_memory() .create() diff --git a/crates/core/src/operations/transaction/state.rs b/crates/core/src/kernel/transaction/state.rs similarity index 98% rename from crates/core/src/operations/transaction/state.rs rename to crates/core/src/kernel/transaction/state.rs index 71251ebd87..5b406b2b70 100644 --- a/crates/core/src/operations/transaction/state.rs +++ b/crates/core/src/kernel/transaction/state.rs @@ -254,6 +254,7 @@ mod tests { use datafusion::prelude::SessionContext; use datafusion_expr::{col, lit}; + use object_store::path::Path; use super::*; use crate::delta_datafusion::{files_matching_predicate, DataFusionMixins}; @@ -269,7 +270,8 @@ mod tests { #[test] fn test_parse_predicate_expression() { - let snapshot = DeltaTableState::from_actions(init_table_actions()).unwrap(); + let snapshot = + DeltaTableState::from_actions(init_table_actions(), &Path::default()).unwrap(); let session = SessionContext::new(); let state = session.state(); @@ -317,7 +319,7 @@ mod tests { true, ))); - let state = DeltaTableState::from_actions(actions).unwrap(); + let state = DeltaTableState::from_actions(actions, &Path::default()).unwrap(); let files = files_matching_predicate(&state.snapshot, &[]) .unwrap() .collect::>(); diff --git a/crates/core/src/lib.rs b/crates/core/src/lib.rs index 5e17a5a26f..a9040139f6 100644 --- a/crates/core/src/lib.rs +++ b/crates/core/src/lib.rs @@ -49,9 +49,10 @@ //! Querying from local filesystem: //! ``` //! use std::sync::Arc; -//! use datafusion::prelude::SessionContext; //! +//! # #[cfg(feature="datafusion")] //! async { +//! use datafusion::prelude::SessionContext; //! let mut ctx = SessionContext::new(); //! let table = deltalake_core::open_table("../test/tests/data/simple_table") //! .await @@ -68,7 +69,6 @@ // #![deny(missing_docs)] #![allow(rustdoc::invalid_html_tags)] #![allow(clippy::nonminimal_bool)] - pub mod data_catalog; pub mod errors; pub mod kernel; @@ -76,10 +76,9 @@ pub mod logstore; pub mod operations; pub mod protocol; pub mod schema; -pub mod storage; pub mod table; -#[cfg(test)] +#[cfg(any(test, feature = "integration_test"))] pub mod test_utils; #[cfg(feature = "datafusion")] @@ -87,6 +86,7 @@ pub mod delta_datafusion; pub mod writer; use std::collections::HashMap; +use std::sync::OnceLock; pub use self::data_catalog::{DataCatalog, DataCatalogError}; pub use self::errors::*; @@ -97,14 +97,17 @@ pub use self::table::config::TableProperty; pub use self::table::DeltaTable; pub use object_store::{path::Path, Error as ObjectStoreError, ObjectMeta, ObjectStore}; pub use operations::DeltaOps; -use std::sync::OnceLock; + +pub use protocol::checkpoints; // convenience exports for consumers to avoid aligning crate versions pub use arrow; #[cfg(feature = "datafusion")] pub use datafusion; pub use parquet; -pub use protocol::checkpoints; + +#[cfg(not(any(feature = "rustls", feature = "native-tls")))] +compile_error!("You must enable at least one of the features: `rustls` or `native-tls`."); /// Creates and loads a DeltaTable from the given path with current metadata. /// Infers the storage backend to use from the scheme in the given table path. @@ -189,7 +192,7 @@ mod tests { let table = crate::open_table("../test/tests/data/delta-0.2.0") .await .unwrap(); - assert_eq!(table.version(), 3); + assert_eq!(table.version(), Some(3)); assert_eq!(table.protocol().unwrap().min_writer_version, 2); assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( @@ -203,7 +206,7 @@ mod tests { let tombstones = table .snapshot() .unwrap() - .all_tombstones(table.object_store().clone()) + .all_tombstones(&table.log_store()) .await .unwrap() .collect_vec(); @@ -242,7 +245,7 @@ mod tests { let mut table = crate::open_table_with_version("../test/tests/data/delta-0.2.0", 0) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.protocol().unwrap().min_writer_version, 2); assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( @@ -256,7 +259,7 @@ mod tests { table = crate::open_table_with_version("../test/tests/data/delta-0.2.0", 2) .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert_eq!(table.protocol().unwrap().min_writer_version, 2); assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( @@ -270,7 +273,7 @@ mod tests { table = crate::open_table_with_version("../test/tests/data/delta-0.2.0", 3) .await .unwrap(); - assert_eq!(table.version(), 3); + assert_eq!(table.version(), Some(3)); assert_eq!(table.protocol().unwrap().min_writer_version, 2); assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( @@ -288,7 +291,7 @@ mod tests { let table = crate::open_table("../test/tests/data/delta-0.8.0") .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.protocol().unwrap().min_writer_version, 2); assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( @@ -320,7 +323,7 @@ mod tests { let tombstones = table .snapshot() .unwrap() - .all_tombstones(table.object_store().clone()) + .all_tombstones(&table.log_store()) .await .unwrap() .collect_vec(); @@ -344,7 +347,7 @@ mod tests { let mut table = crate::open_table("../test/tests/data/delta-0.8.0") .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.protocol().unwrap().min_writer_version, 2); assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( @@ -355,7 +358,7 @@ mod tests { ] ); table.load_version(0).await.unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.protocol().unwrap().min_writer_version, 2); assert_eq!(table.protocol().unwrap().min_reader_version, 1); assert_eq!( @@ -557,16 +560,16 @@ mod tests { async fn test_poll_table_commits() { let path = "../test/tests/data/simple_table_with_checkpoint"; let mut table = crate::open_table_with_version(path, 9).await.unwrap(); - assert_eq!(table.version(), 9); + assert_eq!(table.version(), Some(9)); let peek = table .log_store() - .peek_next_commit(table.version()) + .peek_next_commit(table.version().unwrap()) .await .unwrap(); assert!(matches!(peek, PeekCommit::New(..))); if let PeekCommit::New(version, actions) = peek { - assert_eq!(table.version(), 9); + assert_eq!(table.version(), Some(9)); assert!(!table.get_files_iter().unwrap().any(|f| f == Path::from( "part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet" @@ -577,7 +580,7 @@ mod tests { table.update_incremental(None).await.unwrap(); - assert_eq!(table.version(), 10); + assert_eq!(table.version(), Some(10)); assert!(table.get_files_iter().unwrap().any(|f| f == Path::from( "part-00000-f0e955c5-a1e3-4eec-834e-dcc098fc9005-c000.snappy.parquet" @@ -586,7 +589,7 @@ mod tests { let peek = table .log_store() - .peek_next_commit(table.version()) + .peek_next_commit(table.version().unwrap()) .await .unwrap(); assert!(matches!(peek, PeekCommit::UpToDate)); @@ -596,7 +599,7 @@ mod tests { async fn test_read_vacuumed_log() { let path = "../test/tests/data/checkpoints_vacuumed"; let table = crate::open_table(path).await.unwrap(); - assert_eq!(table.version(), 12); + assert_eq!(table.version(), Some(12)); } #[tokio::test] @@ -649,7 +652,7 @@ mod tests { let table = crate::open_table("../test/tests/data/simple_table_with_cdc") .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert_eq!( table.get_files_iter().unwrap().collect_vec(), vec![Path::from( diff --git a/crates/core/src/logstore/config.rs b/crates/core/src/logstore/config.rs new file mode 100644 index 0000000000..2f001c2a7b --- /dev/null +++ b/crates/core/src/logstore/config.rs @@ -0,0 +1,399 @@ +//! Configuration for the Delta Log Store. +//! +//! This module manages the various pieces of configuration for the Delta Log Store. +//! It provides methods for parsing and updating configuration settings. All configuration +//! is parsed from String -> String mappings. +//! +//! Specific pieces of configuration must implement the `TryUpdateKey` trait which +//! defines how to update internal fields based on key-value pairs. +use std::collections::HashMap; + +use ::object_store::RetryConfig; +use object_store::{path::Path, prefix::PrefixStore, ObjectStore}; + +use super::storage::LimitConfig; +use super::{storage::runtime::RuntimeConfig, IORuntime}; +use crate::{DeltaResult, DeltaTableError}; + +pub trait TryUpdateKey: Default { + /// Update an internal field in the configuration. + /// + /// ## Returns + /// - `Ok(Some(()))` if the key was updated. + /// - `Ok(None)` if the key was not found and no internal field was updated. + /// - `Err(_)` if the update failed. Failed updates may include finding a known key, + /// but failing to parse the value into the expected type. + fn try_update_key(&mut self, key: &str, value: &str) -> DeltaResult>; + + /// Load configuration values from environment variables + /// + /// For Option fields, this will only set values that are None + /// For non-optional fields, environment variables will update the + /// value if the current value corresponds to the default value. + fn load_from_environment(&mut self) -> DeltaResult<()>; +} + +#[derive(Debug)] +/// Generic container for parsing configuration +pub struct ParseResult { + /// Parsed configuration + pub config: T, + /// Unrecognized key value pairs. + pub unparsed: HashMap, + /// Errors encountered during parsing + pub errors: Vec<(String, String)>, + /// Whether the configuration is defaults only - i.e. no custom values were provided + pub is_default: bool, +} + +impl ParseResult { + pub fn raise_errors(&self) -> DeltaResult<()> { + if !self.errors.is_empty() { + return Err(DeltaTableError::Generic(format!( + "Failed to parse config: {:?}", + self.errors + ))); + } + Ok(()) + } +} + +impl FromIterator<(K, V)> for ParseResult +where + T: TryUpdateKey, + K: AsRef + Into, + V: AsRef + Into, +{ + fn from_iter>(iter: I) -> Self { + let mut config = T::default(); + let mut unparsed = HashMap::new(); + let mut errors = Vec::new(); + let mut is_default = true; + for (k, v) in iter { + match config.try_update_key(k.as_ref(), v.as_ref()) { + Ok(None) => { + unparsed.insert(k.into(), v.into()); + } + Ok(Some(_)) => is_default = false, + Err(e) => errors.push((k.into(), e.to_string())), + } + } + ParseResult { + config, + unparsed, + errors, + is_default, + } + } +} + +#[derive(Default, Debug, Clone)] +pub struct StorageConfig { + /// Runtime configuration. + /// + /// Configuration to set up a dedicated IO runtime to execute IO related operations or + /// dedicated handle. + pub runtime: Option, + + pub retry: ::object_store::RetryConfig, + + /// Limit configuration. + /// + /// Configuration to limit the number of concurrent requests to the object store. + pub limit: Option, + + /// Properties that are not recognized by the storage configuration. + /// + /// These properties are ignored by the storage configuration and can be used for custom purposes. + pub unknown_properties: HashMap, + + /// Original unprocessed properties. + /// + /// Since we remove properties during processing, but downstream integrations may + /// use them for their own purposes, we keep a copy of the original properties. + pub raw: HashMap, +} + +impl StorageConfig { + /// Wrap an object store with additional layers of functionality. + /// + /// Depending on the configuration, the following layers may be added: + /// - Retry layer: Adds retry logic to the object store. + /// - Limit layer: Limits the number of concurrent requests to the object store. + pub fn decorate_store( + &self, + store: T, + table_root: &url::Url, + ) -> DeltaResult> { + let inner = Self::decorate_prefix(store, table_root)?; + Ok(inner) + } + + pub(crate) fn decorate_prefix( + store: T, + table_root: &url::Url, + ) -> DeltaResult> { + let prefix = super::object_store_path(table_root)?; + Ok(if prefix != Path::from("/") { + Box::new(PrefixStore::new(store, prefix)) as Box + } else { + Box::new(store) as Box + }) + } +} + +impl FromIterator<(K, V)> for StorageConfig +where + K: AsRef + Into, + V: AsRef + Into, +{ + fn from_iter>(iter: I) -> Self { + let mut config = StorageConfig { + raw: iter + .into_iter() + .map(|(k, v)| (k.into(), v.into())) + .collect(), + ..Default::default() + }; + + let result = ParseResult::::from_iter(&config.raw); + if let Some(runtime_config) = (!result.is_default).then_some(result.config) { + config.runtime = Some(IORuntime::Config(runtime_config)); + }; + + let result = ParseResult::::from_iter(result.unparsed); + config.limit = (!result.is_default).then_some(result.config); + + let remainder = result.unparsed; + + let remainder = { + let result = ParseResult::::from_iter(remainder); + config.retry = result.config; + result.unparsed + }; + + config.unknown_properties = remainder; + config + } +} + +impl StorageConfig { + pub fn raw(&self) -> impl Iterator { + self.raw.iter() + } + + /// Parse options into a StorageConfig. + /// + /// This method will raise if it cannot parse a value. StorageConfig can also + /// be constructed from an iterator of key-value pairs which will ignore any + /// parsing errors. + /// + /// # Raises + /// + /// Raises a `DeltaError` if any of the options are invalid - i.e. cannot be parsed into target type. + pub fn parse_options(options: I) -> DeltaResult + where + I: IntoIterator, + K: AsRef + Into, + V: AsRef + Into, + { + let mut props = StorageConfig { + raw: options + .into_iter() + .map(|(k, v)| (k.into(), v.into())) + .collect(), + ..Default::default() + }; + + let (runtime, remainder): (RuntimeConfig, _) = try_parse_impl(&props.raw)?; + // NOTE: we only want to assign an actual runtime config we consumed an option + if props.raw.len() > remainder.len() { + props.runtime = Some(IORuntime::Config(runtime)); + } + + let result = ParseResult::::from_iter(remainder); + result.raise_errors()?; + props.limit = (!result.is_default).then_some(result.config); + let remainder = result.unparsed; + + let remainder = { + let (retry, remainder): (RetryConfig, _) = try_parse_impl(remainder)?; + props.retry = retry; + remainder + }; + + props.unknown_properties = remainder; + Ok(props) + } + + // Provide an IO Runtime directly + pub fn with_io_runtime(mut self, rt: IORuntime) -> Self { + self.runtime = Some(rt); + self + } +} + +pub(super) fn try_parse_impl( + options: I, +) -> DeltaResult<(T, HashMap)> +where + I: IntoIterator, + K: AsRef + Into, + V: AsRef + Into, + T: TryUpdateKey, +{ + let result = ParseResult::from_iter(options); + result.raise_errors()?; + Ok((result.config, result.unparsed)) +} + +pub fn parse_usize(value: &str) -> DeltaResult { + value + .parse::() + .map_err(|_| DeltaTableError::Generic(format!("failed to parse \"{value}\" as usize"))) +} + +pub fn parse_f64(value: &str) -> DeltaResult { + value + .parse::() + .map_err(|_| DeltaTableError::Generic(format!("failed to parse \"{value}\" as f64"))) +} + +pub fn parse_duration(value: &str) -> DeltaResult { + humantime::parse_duration(value) + .map_err(|_| DeltaTableError::Generic(format!("failed to parse \"{value}\" as Duration"))) +} + +pub fn parse_bool(value: &str) -> DeltaResult { + Ok(str_is_truthy(value)) +} + +pub fn parse_string(value: &str) -> DeltaResult { + Ok(value.to_string()) +} + +/// Return true for all the stringly values typically associated with true +/// +/// aka YAML booleans +/// +/// ```rust +/// # use deltalake_core::logstore::config::*; +/// for value in ["1", "true", "on", "YES", "Y"] { +/// assert!(str_is_truthy(value)); +/// } +/// for value in ["0", "FALSE", "off", "NO", "n", "bork"] { +/// assert!(!str_is_truthy(value)); +/// } +/// ``` +pub fn str_is_truthy(val: &str) -> bool { + val.eq_ignore_ascii_case("1") + | val.eq_ignore_ascii_case("true") + | val.eq_ignore_ascii_case("on") + | val.eq_ignore_ascii_case("yes") + | val.eq_ignore_ascii_case("y") +} + +#[cfg(test)] +mod tests { + use super::*; + use maplit::hashmap; + use object_store::RetryConfig; + use std::time::Duration; + + // Test retry config parsing + #[test] + fn test_retry_config_from_options() { + let options = hashmap! { + "max_retries".to_string() => "100".to_string() , + "retry_timeout".to_string() => "300s".to_string() , + "backoff_config.init_backoff".to_string() => "20s".to_string() , + "backoff_config.max_backoff".to_string() => "1h".to_string() , + "backoff_config.base".to_string() => "50.0".to_string() , + }; + let (retry_config, remainder): (RetryConfig, _) = super::try_parse_impl(options).unwrap(); + assert!(remainder.is_empty()); + + assert_eq!(retry_config.max_retries, 100); + assert_eq!(retry_config.retry_timeout, Duration::from_secs(300)); + assert_eq!(retry_config.backoff.init_backoff, Duration::from_secs(20)); + assert_eq!(retry_config.backoff.max_backoff, Duration::from_secs(3600)); + assert_eq!(retry_config.backoff.base, 50_f64); + } + + // Test ParseResult functionality + #[test] + fn test_parse_result_handling() { + let options = hashmap! { + "retry_timeout".to_string() => "300s".to_string(), + "max_retries".to_string() => "not_a_number".to_string(), + "unknown_key".to_string() => "value".to_string(), + }; + + let result: ParseResult = options.into_iter().collect(); + println!("result: {:?}", result); + assert!(!result.errors.is_empty()); + assert!(!result.unparsed.is_empty()); + assert!(!result.is_default); + + assert!(result.raise_errors().is_err()); + } + + // Test StorageConfig parsing + #[test] + fn test_storage_config_parsing() { + let options = hashmap! { + "max_retries".to_string() => "5".to_string(), + "retry_timeout".to_string() => "10s".to_string(), + "unknown_prop".to_string() => "value".to_string(), + }; + + let config = StorageConfig::parse_options(options).unwrap(); + assert_eq!(config.retry.max_retries, 5); + assert_eq!(config.retry.retry_timeout, Duration::from_secs(10)); + assert!(config.unknown_properties.contains_key("unknown_prop")); + } + + // Test utility parsing functions + #[test] + #[allow(clippy::approx_constant)] + fn test_parsing_utilities() { + assert_eq!(parse_usize("42").unwrap(), 42); + assert!(parse_usize("not_a_number").is_err()); + + assert_eq!(parse_f64("3.14").unwrap(), 3.14); + assert!(parse_f64("not_a_number").is_err()); + + assert_eq!(parse_duration("1h").unwrap(), Duration::from_secs(3600)); + assert!(parse_duration("invalid").is_err()); + + assert!(parse_bool("true").unwrap()); + assert!(parse_bool("1").unwrap()); + assert!(!parse_bool("false").unwrap()); + assert!(!parse_bool("0").unwrap()); + + assert_eq!(parse_string("test").unwrap(), "test"); + } + + // Test str_is_truthy function + #[test] + fn test_str_is_truthy() { + let truthy_values = ["1", "true", "on", "YES", "Y", "True", "ON"]; + let falsy_values = ["0", "false", "off", "NO", "n", "bork", "False", "OFF"]; + + for value in truthy_values { + assert!(str_is_truthy(value), "{} should be truthy", value); + } + + for value in falsy_values { + assert!(!str_is_truthy(value), "{} should be falsy", value); + } + } + + // Test StorageConfig with IO runtime + #[test] + fn test_storage_config_with_io_runtime() { + let config = + StorageConfig::default().with_io_runtime(IORuntime::Config(RuntimeConfig::default())); + assert!(config.runtime.is_some()); + } +} diff --git a/crates/core/src/logstore/default_logstore.rs b/crates/core/src/logstore/default_logstore.rs index e5c3a54e53..ed9d38fae2 100644 --- a/crates/core/src/logstore/default_logstore.rs +++ b/crates/core/src/logstore/default_logstore.rs @@ -6,12 +6,10 @@ use bytes::Bytes; use object_store::{Attributes, Error as ObjectStoreError, ObjectStore, PutOptions, TagSet}; use uuid::Uuid; +use super::storage::{utils::commit_uri_from_version, ObjectStoreRef}; use super::{CommitOrBytes, LogStore, LogStoreConfig}; -use crate::{ - operations::transaction::TransactionError, - storage::{commit_uri_from_version, ObjectStoreRef}, - DeltaResult, -}; +use crate::kernel::transaction::TransactionError; +use crate::DeltaResult; fn put_options() -> &'static PutOptions { static PUT_OPTS: OnceLock = OnceLock::new(); @@ -19,13 +17,15 @@ fn put_options() -> &'static PutOptions { mode: object_store::PutMode::Create, // Creates if file doesn't exists yet tags: TagSet::default(), attributes: Attributes::default(), + extensions: Default::default(), }) } /// Default [`LogStore`] implementation #[derive(Debug, Clone)] pub struct DefaultLogStore { - pub(crate) storage: ObjectStoreRef, + prefixed_store: ObjectStoreRef, + root_store: ObjectStoreRef, config: LogStoreConfig, } @@ -34,10 +34,21 @@ impl DefaultLogStore { /// /// # Arguments /// - /// * `storage` - A shared reference to an [`object_store::ObjectStore`] with "/" pointing at delta table root (i.e. where `_delta_log` is located). + /// * `prefixed_store` - A shared reference to an [`object_store::ObjectStore`] with "/" + /// pointing at delta table root (i.e. where `_delta_log` is located). + /// * `root_store` - A shared reference to an [`object_store::ObjectStore`] with "/" + /// pointing at root of the storage system. /// * `location` - A url corresponding to the storage location of `storage`. - pub fn new(storage: ObjectStoreRef, config: LogStoreConfig) -> Self { - Self { storage, config } + pub fn new( + prefixed_store: ObjectStoreRef, + root_store: ObjectStoreRef, + config: LogStoreConfig, + ) -> Self { + Self { + prefixed_store, + root_store, + config, + } } } @@ -100,12 +111,12 @@ impl LogStore for DefaultLogStore { super::get_latest_version(self, current_version).await } - async fn get_earliest_version(&self, current_version: i64) -> DeltaResult { - super::get_earliest_version(self, current_version).await + fn object_store(&self, _: Option) -> Arc { + self.prefixed_store.clone() } - fn object_store(&self, _: Option) -> Arc { - self.storage.clone() + fn root_object_store(&self, _: Option) -> Arc { + self.root_store.clone() } fn config(&self) -> &LogStoreConfig { diff --git a/crates/core/src/logstore/factories.rs b/crates/core/src/logstore/factories.rs new file mode 100644 index 0000000000..b7a6cc7e24 --- /dev/null +++ b/crates/core/src/logstore/factories.rs @@ -0,0 +1,165 @@ +use std::{ + collections::HashMap, + sync::{Arc, OnceLock}, +}; + +use dashmap::DashMap; +use object_store::RetryConfig; +use object_store::{path::Path, DynObjectStore}; +use tokio::runtime::Handle; +use url::Url; + +use super::{default_logstore, DeltaIOStorageBackend, LogStore, ObjectStoreRef, StorageConfig}; +use crate::{DeltaResult, DeltaTableError}; + +/// Factory registry to manage [`ObjectStoreFactory`] instances +pub type ObjectStoreFactoryRegistry = Arc>>; + +/// Factory trait for creating [`ObjectStore`](::object_store::ObjectStore) instances at runtime +pub trait ObjectStoreFactory: Send + Sync { + /// Parse URL options and create an object store instance. + /// + /// The object store instance returned by this method must point at the root of the storage location. + /// Root in this case means scheme, authority/host and maybe port. + /// The path segment is returned as second element of the tuple. It must point at the path + /// corresponding to the path segment of the URL. + /// + /// The store should __NOT__ apply the decorations via the passed `options` + fn parse_url_opts( + &self, + url: &Url, + options: &HashMap, + retry: &RetryConfig, + handle: Option, + ) -> DeltaResult<(ObjectStoreRef, Path)>; +} + +#[derive(Clone, Debug, Default)] +pub(crate) struct DefaultObjectStoreFactory {} + +impl ObjectStoreFactory for DefaultObjectStoreFactory { + fn parse_url_opts( + &self, + url: &Url, + options: &HashMap, + _retry: &RetryConfig, + handle: Option, + ) -> DeltaResult<(ObjectStoreRef, Path)> { + let (mut store, path) = default_parse_url_opts(url, options)?; + + if let Some(handle) = handle { + store = Arc::new(DeltaIOStorageBackend::new(store, handle)) as Arc; + } + Ok((store, path)) + } +} + +fn default_parse_url_opts( + url: &Url, + options: &HashMap, +) -> DeltaResult<(ObjectStoreRef, Path)> { + match url.scheme() { + "memory" | "file" => { + let (store, path) = object_store::parse_url_opts(url, options)?; + tracing::debug!("building store with:\n\tParsed URL: {url}\n\tPath in store: {path}"); + Ok((Arc::new(store), path)) + } + _ => Err(DeltaTableError::InvalidTableLocation(url.clone().into())), + } +} + +/// Access global registry of object store factories +pub fn object_store_factories() -> ObjectStoreFactoryRegistry { + static REGISTRY: OnceLock = OnceLock::new(); + let factory = Arc::new(DefaultObjectStoreFactory::default()); + REGISTRY + .get_or_init(|| { + let registry = ObjectStoreFactoryRegistry::default(); + registry.insert(Url::parse("memory://").unwrap(), factory.clone()); + registry.insert(Url::parse("file://").unwrap(), factory); + registry + }) + .clone() +} + +/// Simpler access pattern for the [ObjectStoreFactoryRegistry] to get a single store +pub fn store_for(url: &Url, options: I) -> DeltaResult +where + I: IntoIterator, + K: AsRef + Into, + V: AsRef + Into, +{ + let scheme = Url::parse(&format!("{}://", url.scheme())).unwrap(); + let storage_config = StorageConfig::parse_options(options)?; + if let Some(factory) = object_store_factories().get(&scheme) { + let (store, _prefix) = + factory.parse_url_opts(url, &storage_config.raw, &storage_config.retry, None)?; + let store = storage_config.decorate_store(store, url)?; + Ok(Arc::new(store)) + } else { + Err(DeltaTableError::InvalidTableLocation(url.clone().into())) + } +} + +/// Registry of [`LogStoreFactory`] instances +pub type LogStoreFactoryRegistry = Arc>>; + +/// Trait for generating [LogStore] implementations +pub trait LogStoreFactory: Send + Sync { + /// Create a new [`LogStore`] from options. + /// + /// This method is responsible for creating a new instance of the [LogStore] implementation. + /// + /// ## Parameters + /// - `prefixed_store`: A reference to the object store. + /// - `location`: A reference to the URL of the location. + /// - `options`: A reference to the storage configuration options. + /// + /// It returns a [DeltaResult] containing an [Arc] to the newly created [LogStore] implementation. + fn with_options( + &self, + prefixed_store: ObjectStoreRef, + root_store: ObjectStoreRef, + location: &Url, + options: &StorageConfig, + ) -> DeltaResult>; +} + +#[derive(Clone, Debug, Default)] +struct DefaultLogStoreFactory {} + +impl LogStoreFactory for DefaultLogStoreFactory { + fn with_options( + &self, + prefixed_store: ObjectStoreRef, + root_store: ObjectStoreRef, + location: &Url, + options: &StorageConfig, + ) -> DeltaResult> { + Ok(default_logstore( + prefixed_store, + root_store, + location, + options, + )) + } +} + +/// Access global registry of logstore factories. +pub fn logstore_factories() -> LogStoreFactoryRegistry { + static REGISTRY: OnceLock = OnceLock::new(); + REGISTRY + .get_or_init(|| { + let registry = LogStoreFactoryRegistry::default(); + registry.insert( + Url::parse("memory://").unwrap(), + Arc::new(DefaultLogStoreFactory::default()), + ); + registry.insert( + Url::parse("file://").unwrap(), + Arc::new(DefaultLogStoreFactory::default()), + ); + registry + }) + .clone() +} diff --git a/crates/core/src/logstore/mod.rs b/crates/core/src/logstore/mod.rs index 6a1e826a23..9fe5ffd2a6 100644 --- a/crates/core/src/logstore/mod.rs +++ b/crates/core/src/logstore/mod.rs @@ -1,60 +1,154 @@ -//! Delta log store. -use std::cmp::min; +//! # DeltaLake storage system +//! +//! Interacting with storage systems is a crucial part of any table format. +//! On one had the storage abstractions need to provide certain guarantees +//! (e.g. atomic rename, ...) and meet certain assumptions (e.g. sorted list results) +//! on the other hand can we exploit our knowledge about the general file layout +//! and access patterns to optimize our operations in terms of cost and performance. +//! +//! Two distinct phases are involved in querying a Delta table: +//! - **Metadata**: Fetching metadata about the table, such as schema, partitioning, and statistics. +//! - **Data**: Reading and processing data files based on the metadata. +//! +//! When writing to a table, we see the same phases, just in inverse order: +//! - **Data**: Writing data files that should become part of the table. +//! - **Metadata**: Updating table metadata to incorporate updates. +//! +//! Two main abstractions govern the file operations [`LogStore`] and [`ObjectStore`]. +//! +//! [`LogStore`]s are scoped to individual tables and are responsible for maintaining proper +//! behaviours and ensuring consistency during the metadata phase. The correctness is predicated +//! on the atomicity and durability guarantees of the implementation of this interface. +//! +//! - Atomic visibility: Partial writes must not be visible to readers. +//! - Mutual exclusion: Only one writer must be able to write to a specific log file. +//! - Consistent listing: Once a file has been written, any future list files operation must return +//! the underlying file system entry must immediately. +//! +//!
+//! +//! While most object stores today provide the required guarantees, the specific +//! locking mechanics are a table level responsibility. Specific implementations may +//! decide to refer to a central catalog or other mechanisms for coordination. +//! +//!
+//! +//! [`ObjectStore`]s are responsible for direct interactions with storage systems. Either +//! during the data phase, where additional requirements are imposed on the storage system, +//! or by specific LogStore implementations for their internal object store interactions. +//! +//! ## Managing LogStores and ObjectStores. +//! +//! Aside from very basic implementations (i.e. in-memory and local file system) we rely +//! on external integrations to provide [`ObjectStore`] and/or [`LogStore`] implementations. +//! +//! At runtime, deltalake needs to produce appropriate [`ObjectStore`]s to access the files +//! discovered in a table. This is done via +//! +//! ## Configuration +//! +use std::collections::HashMap; use std::io::{BufRead, BufReader, Cursor}; -use std::sync::{LazyLock, OnceLock}; -use std::{cmp::max, collections::HashMap, sync::Arc}; +use std::sync::{Arc, LazyLock}; use bytes::Bytes; -use dashmap::DashMap; -use delta_kernel::AsAny; -use futures::{StreamExt, TryStreamExt}; +#[cfg(feature = "datafusion")] +use datafusion::datasource::object_store::ObjectStoreUrl; +use delta_kernel::engine::default::executor::tokio::{ + TokioBackgroundExecutor, TokioMultiThreadExecutor, +}; +use delta_kernel::engine::default::DefaultEngine; +use delta_kernel::log_segment::LogSegment; +use delta_kernel::path::{LogPathFileType, ParsedLogPath}; +use delta_kernel::{AsAny, Engine}; +use futures::StreamExt; +use object_store::ObjectStoreScheme; use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; use regex::Regex; use serde::de::{Error, SeqAccess, Visitor}; use serde::ser::SerializeSeq; use serde::{Deserialize, Serialize}; -use tracing::{debug, warn}; +use tokio::runtime::RuntimeFlavor; +use tokio::task::spawn_blocking; +use tracing::{debug, error}; use url::Url; use uuid::Uuid; -use crate::kernel::log_segment::PathExt; +use crate::kernel::transaction::TransactionError; use crate::kernel::Action; -use crate::operations::transaction::TransactionError; -use crate::protocol::{get_last_checkpoint, ProtocolError}; -use crate::storage::DeltaIOStorageBackend; -use crate::storage::{ - commit_uri_from_version, retry_ext::ObjectStoreRetryExt, IORuntime, ObjectStoreRef, - StorageOptions, -}; - use crate::{DeltaResult, DeltaTableError}; -#[cfg(feature = "datafusion")] -use datafusion::datasource::object_store::ObjectStoreUrl; +pub use self::config::StorageConfig; +pub use self::factories::{ + logstore_factories, object_store_factories, store_for, LogStoreFactory, + LogStoreFactoryRegistry, ObjectStoreFactory, ObjectStoreFactoryRegistry, +}; +pub use self::storage::utils::commit_uri_from_version; +pub use self::storage::{ + DefaultObjectStoreRegistry, DeltaIOStorageBackend, IORuntime, ObjectStoreRef, + ObjectStoreRegistry, ObjectStoreRetryExt, +}; +/// Convenience re-export of the object store crate +pub use ::object_store; +pub mod config; pub(crate) mod default_logstore; +pub(crate) mod factories; +pub(crate) mod storage; + +/// Internal trait to handle object store configuration and initialization. +trait LogStoreFactoryExt { + /// Create a new log store with the given options. + /// + /// ## Parameters + /// + /// - `root_store`: and instance of [`ObjectStoreRef`] with no prefix o.a. applied. + /// I.e. pointing to the root of the onject store. + /// - `location`: The location of the the delta table (where the `_delta_log` directory is). + /// - `options`: The options for the log store. + fn with_options_internal( + &self, + root_store: ObjectStoreRef, + location: &Url, + options: &StorageConfig, + ) -> DeltaResult; +} + +impl LogStoreFactoryExt for T { + fn with_options_internal( + &self, + root_store: ObjectStoreRef, + location: &Url, + options: &StorageConfig, + ) -> DeltaResult { + let prefixed_store = options.decorate_store(root_store.clone(), location)?; + let log_store = + self.with_options(Arc::new(prefixed_store), root_store, location, options)?; + Ok(log_store) + } +} -/// Trait for generating [LogStore] implementations -pub trait LogStoreFactory: Send + Sync { - /// Create a new [LogStore] - fn with_options( +impl LogStoreFactoryExt for Arc { + fn with_options_internal( &self, - store: ObjectStoreRef, + root_store: ObjectStoreRef, location: &Url, - options: &StorageOptions, - ) -> DeltaResult> { - Ok(default_logstore(store, location, options)) + options: &StorageConfig, + ) -> DeltaResult { + T::with_options_internal(self, root_store, location, options) } } /// Return the [DefaultLogStore] implementation with the provided configuration options pub fn default_logstore( - store: ObjectStoreRef, + prefixed_store: ObjectStoreRef, + root_store: ObjectStoreRef, location: &Url, - options: &StorageOptions, + options: &StorageConfig, ) -> Arc { Arc::new(default_logstore::DefaultLogStore::new( - store, + prefixed_store, + root_store, LogStoreConfig { location: location.clone(), options: options.clone(), @@ -62,32 +156,6 @@ pub fn default_logstore( )) } -#[derive(Clone, Debug, Default)] -struct DefaultLogStoreFactory {} -impl LogStoreFactory for DefaultLogStoreFactory {} - -/// Registry of [LogStoreFactory] instances -pub type FactoryRegistry = Arc>>; - -/// TODO -pub fn logstores() -> FactoryRegistry { - static REGISTRY: OnceLock = OnceLock::new(); - REGISTRY - .get_or_init(|| { - let registry = FactoryRegistry::default(); - registry.insert( - Url::parse("memory://").unwrap(), - Arc::new(DefaultLogStoreFactory::default()), - ); - registry.insert( - Url::parse("file://").unwrap(), - Arc::new(DefaultLogStoreFactory::default()), - ); - registry - }) - .clone() -} - /// Sharable reference to [`LogStore`] pub type LogStoreRef = Arc; @@ -102,51 +170,45 @@ static DELTA_LOG_PATH: LazyLock = LazyLock::new(|| Path::from("_delta_log" /// # use std::collections::HashMap; /// # use url::Url; /// let location = Url::parse("memory:///").expect("Failed to make location"); -/// let logstore = logstore_for(location, HashMap::new(), None).expect("Failed to get a logstore"); +/// let storage_config = StorageConfig::default(); +/// let logstore = logstore_for(location, storage_config).expect("Failed to get a logstore"); /// ``` -pub fn logstore_for( - location: Url, - options: impl Into + Clone, - io_runtime: Option, -) -> DeltaResult { +pub fn logstore_for(location: Url, storage_config: StorageConfig) -> DeltaResult { // turn location into scheme let scheme = Url::parse(&format!("{}://", location.scheme())) .map_err(|_| DeltaTableError::InvalidTableLocation(location.clone().into()))?; - if let Some(entry) = crate::storage::factories().get(&scheme) { + if let Some(entry) = object_store_factories().get(&scheme) { debug!("Found a storage provider for {scheme} ({location})"); - - let (store, _prefix) = entry - .value() - .parse_url_opts(&location, &options.clone().into())?; - return logstore_with(store, location, options, io_runtime); + let (root_store, _prefix) = entry.value().parse_url_opts( + &location, + &storage_config.raw, + &storage_config.retry, + storage_config.runtime.clone().map(|rt| rt.get_handle()), + )?; + return logstore_with(root_store, location, storage_config); } + Err(DeltaTableError::InvalidTableLocation(location.into())) } /// Return the [LogStoreRef] using the given [ObjectStoreRef] pub fn logstore_with( - store: ObjectStoreRef, + root_store: ObjectStoreRef, location: Url, - options: impl Into + Clone, - io_runtime: Option, + storage_config: StorageConfig, ) -> DeltaResult { let scheme = Url::parse(&format!("{}://", location.scheme())) .map_err(|_| DeltaTableError::InvalidTableLocation(location.clone().into()))?; - let store = if let Some(io_runtime) = io_runtime { - Arc::new(DeltaIOStorageBackend::new(store, io_runtime.get_handle())) as ObjectStoreRef - } else { - store - }; - - if let Some(factory) = logstores().get(&scheme) { + if let Some(factory) = logstore_factories().get(&scheme) { debug!("Found a logstore provider for {scheme}"); - return factory.with_options(store, &location, &options.into()); - } else { - println!("Could not find a logstore for the scheme {scheme}"); - warn!("Could not find a logstore for the scheme {scheme}"); + return factory + .value() + .with_options_internal(root_store, &location, &storage_config); } + + error!("Could not find a logstore for the scheme {scheme}"); Err(DeltaTableError::InvalidTableLocation( location.clone().into(), )) @@ -176,8 +238,23 @@ pub enum PeekCommit { pub struct LogStoreConfig { /// url corresponding to the storage location. pub location: Url, - /// Options used for configuring backend storage - pub options: StorageOptions, + // Options used for configuring backend storage + pub options: StorageConfig, +} + +impl LogStoreConfig { + pub fn decorate_store( + &self, + store: T, + table_root: Option<&url::Url>, + ) -> DeltaResult> { + let table_url = table_root.unwrap_or(&self.location); + self.options.decorate_store(store, table_url) + } + + pub fn object_store_factory(&self) -> ObjectStoreFactoryRegistry { + self::factories::object_store_factories() + } } /// Trait for critical operations required to read and write commit entries in Delta logs. @@ -225,9 +302,6 @@ pub trait LogStore: Send + Sync + AsAny { /// Find latest version currently stored in the delta log. async fn get_latest_version(&self, start_version: i64) -> DeltaResult; - /// Find earliest version currently stored in the delta log. - async fn get_earliest_version(&self, start_version: i64) -> DeltaResult; - /// Get the list of actions for the next commit async fn peek_next_commit(&self, current_version: i64) -> DeltaResult { let next_version = current_version + 1; @@ -244,6 +318,13 @@ pub trait LogStore: Send + Sync + AsAny { /// Get object store, can pass operation_id for object stores linked to an operation fn object_store(&self, operation_id: Option) -> Arc; + fn root_object_store(&self, operation_id: Option) -> Arc; + + async fn engine(&self, operation_id: Option) -> Arc { + let store = self.root_object_store(operation_id); + get_engine(store).await + } + /// [Path] to Delta log fn to_uri(&self, location: &Path) -> String { let root = &self.config().location; @@ -260,30 +341,38 @@ pub trait LogStore: Send + Sync + AsAny { &DELTA_LOG_PATH } + #[deprecated( + since = "0.1.0", + note = "DO NOT USE: Just a stop grap to support lakefs during kernel migration" + )] + fn transaction_url(&self, _operation_id: Uuid, base: &Url) -> DeltaResult { + Ok(base.clone()) + } + /// Check if the location is a delta table location async fn is_delta_table_location(&self) -> DeltaResult { let object_store = self.object_store(None); - let mut stream = object_store.list(Some(self.log_path())); + let dummy_url = Url::parse("http://example.com").unwrap(); + let log_path = Path::from("_delta_log"); + + let mut stream = object_store.list(Some(&log_path)); while let Some(res) = stream.next().await { match res { Ok(meta) => { - // Valid but optional files. - if meta.location.is_crc_file() - || meta.location.is_last_checkpoint_file() - || meta.location.is_last_vacuum_info_file() - || meta.location.is_deletion_vector_file() - { - continue; - } - let is_valid = - meta.location.is_commit_file() || meta.location.is_checkpoint_file(); - if !is_valid { - warn!( - "Expected a valid delta file. Found {}", - meta.location.filename().unwrap_or("") - ) + let file_url = dummy_url.join(meta.location.as_ref()).unwrap(); + if let Ok(Some(parsed_path)) = ParsedLogPath::try_from(file_url) { + if matches!( + parsed_path.file_type, + LogPathFileType::Commit + | LogPathFileType::SinglePartCheckpoint + | LogPathFileType::UuidCheckpoint(_) + | LogPathFileType::MultiPartCheckpoint { .. } + | LogPathFileType::CompactedCommit { .. } + ) { + return Ok(true); + } } - return Ok(is_valid); + continue; } Err(ObjectStoreError::NotFound { .. }) => return Ok(false), Err(err) => return Err(err.into()), @@ -293,6 +382,9 @@ pub trait LogStore: Send + Sync + AsAny { Ok(false) } + /// Get configuration representing configured log store. + fn config(&self) -> &LogStoreConfig; + #[cfg(feature = "datafusion")] /// Generate a unique enough url to identify the store in datafusion. /// The DF object store registry only cares about the scheme and the host of the url for @@ -302,9 +394,124 @@ pub trait LogStore: Send + Sync + AsAny { fn object_store_url(&self) -> ObjectStoreUrl { crate::logstore::object_store_url(&self.config().location) } +} - /// Get configuration representing configured log store. - fn config(&self) -> &LogStoreConfig; +/// Extension trait for LogStore to handle some internal invariants. +pub(crate) trait LogStoreExt: LogStore { + /// The the fully qualified table URL + /// + /// The paths is guaranteed to end with a slash, + /// so that it can be used as a prefix for other paths. + fn table_root_url(&self) -> Url { + let mut base = self.config().location.clone(); + if !base.path().ends_with("/") { + base.set_path(&format!("{}/", base.path())); + } + base + } + + /// The the fully qualified table log URL + /// + /// The paths is guaranteed to end with a slash, + /// so that it can be used as a prefix for other paths. + fn log_root_url(&self) -> Url { + self.table_root_url().join("_delta_log/").unwrap() + } +} + +impl LogStoreExt for T {} + +#[async_trait::async_trait] +impl LogStore for Arc { + fn name(&self) -> String { + T::name(self) + } + + async fn refresh(&self) -> DeltaResult<()> { + T::refresh(self).await + } + + async fn read_commit_entry(&self, version: i64) -> DeltaResult> { + T::read_commit_entry(self, version).await + } + + async fn write_commit_entry( + &self, + version: i64, + commit_or_bytes: CommitOrBytes, + operation_id: Uuid, + ) -> Result<(), TransactionError> { + T::write_commit_entry(self, version, commit_or_bytes, operation_id).await + } + + async fn abort_commit_entry( + &self, + version: i64, + commit_or_bytes: CommitOrBytes, + operation_id: Uuid, + ) -> Result<(), TransactionError> { + T::abort_commit_entry(self, version, commit_or_bytes, operation_id).await + } + + async fn get_latest_version(&self, start_version: i64) -> DeltaResult { + T::get_latest_version(self, start_version).await + } + + async fn peek_next_commit(&self, current_version: i64) -> DeltaResult { + T::peek_next_commit(self, current_version).await + } + + fn object_store(&self, operation_id: Option) -> Arc { + T::object_store(self, operation_id) + } + + fn root_object_store(&self, operation_id: Option) -> Arc { + T::root_object_store(self, operation_id) + } + + async fn engine(&self, operation_id: Option) -> Arc { + T::engine(self, operation_id).await + } + + fn to_uri(&self, location: &Path) -> String { + T::to_uri(self, location) + } + + fn root_uri(&self) -> String { + T::root_uri(self) + } + + fn log_path(&self) -> &Path { + T::log_path(self) + } + + async fn is_delta_table_location(&self) -> DeltaResult { + T::is_delta_table_location(self).await + } + + fn config(&self) -> &LogStoreConfig { + T::config(self) + } + + #[cfg(feature = "datafusion")] + fn object_store_url(&self) -> ObjectStoreUrl { + T::object_store_url(self) + } +} + +async fn get_engine(store: Arc) -> Arc { + let handle = tokio::runtime::Handle::current(); + match handle.runtime_flavor() { + RuntimeFlavor::MultiThread => Arc::new(DefaultEngine::new( + store, + Arc::new(TokioMultiThreadExecutor::new(handle)), + )), + RuntimeFlavor::CurrentThread => Arc::new(DefaultEngine::new( + store, + Arc::new(TokioBackgroundExecutor::new()), + )), + _ => panic!("unsupported runtime flavor"), + } } #[cfg(feature = "datafusion")] @@ -319,6 +526,16 @@ fn object_store_url(location: &Url) -> ObjectStoreUrl { .expect("Invalid object store url.") } +/// Parse the path from a URL accounting for special case witjh S3 +// TODO: find out why this is necessary +pub(crate) fn object_store_path(table_root: &Url) -> DeltaResult { + Ok(match ObjectStoreScheme::parse(table_root) { + Ok((ObjectStoreScheme::AmazonS3, _)) => Path::parse(table_root.path())?, + Ok((_, path)) => path, + _ => Path::parse(table_root.path())?, + }) +} + /// TODO pub fn to_uri(root: &Url, location: &Path) -> String { match root.scheme() { @@ -385,7 +602,7 @@ impl Serialize for LogStoreConfig { { let mut seq = serializer.serialize_seq(None)?; seq.serialize_element(&self.location.to_string())?; - seq.serialize_element(&self.options.0)?; + seq.serialize_element(&self.options.raw)?; seq.end() } } @@ -414,10 +631,10 @@ impl<'de> Deserialize<'de> for LogStoreConfig { let options: HashMap = seq .next_element()? .ok_or_else(|| A::Error::invalid_length(0, &self))?; - let location = Url::parse(&location_str).unwrap(); + let location = Url::parse(&location_str).map_err(A::Error::custom)?; Ok(LogStoreConfig { location, - options: options.into(), + options: StorageConfig::parse_options(options).map_err(A::Error::custom)?, }) } } @@ -441,103 +658,31 @@ pub async fn get_latest_version( log_store: &dyn LogStore, current_version: i64, ) -> DeltaResult { - let version_start = match get_last_checkpoint(log_store).await { - Ok(last_check_point) => last_check_point.version, - Err(ProtocolError::CheckpointNotFound) => -1, // no checkpoint - Err(e) => return Err(DeltaTableError::from(e)), - }; - - debug!("latest checkpoint version: {version_start}"); - - let version_start = max(current_version, version_start); - - // list files to find max version - let version = async { - let mut max_version: i64 = version_start; - let prefix = Some(log_store.log_path()); - let offset_path = commit_uri_from_version(max_version); - let object_store = log_store.object_store(None); - let mut files = object_store.list_with_offset(prefix, &offset_path); - let mut empty_stream = true; - - while let Some(obj_meta) = files.next().await { - let obj_meta = obj_meta?; - if let Some(log_version) = extract_version_from_filename(obj_meta.location.as_ref()) { - max_version = max(max_version, log_version); - // also cache timestamp for version, for faster time-travel - // TODO: temporarily disabled because `version_timestamp` is not available in the [`LogStore`] - // self.version_timestamp - // .insert(log_version, obj_meta.last_modified.timestamp()); - } - empty_stream = false; - } - - if max_version < 0 { - return Err(DeltaTableError::not_a_table(log_store.root_uri())); - } - - // This implies no files were fetched during list_offset so either the starting_version is the latest - // or starting_version is invalid, so we use current_version -1, and do one more try. - if empty_stream { - let obj_meta = object_store - .head(&commit_uri_from_version(max_version)) - .await; - if obj_meta.is_err() { - return Box::pin(get_latest_version(log_store, -1)).await; - } - } - - Ok::(max_version) - } - .await?; - - Ok(version) -} - -/// Default implementation for retrieving the earliest version -pub async fn get_earliest_version( - log_store: &dyn LogStore, - current_version: i64, -) -> DeltaResult { - let version_start = match get_last_checkpoint(log_store).await { - Ok(last_check_point) => last_check_point.version, - Err(ProtocolError::CheckpointNotFound) => { - // no checkpoint so start from current_version - current_version - } - Err(e) => { - return Err(DeltaTableError::from(e)); - } + let current_version = if current_version < 0 { + 0 + } else { + current_version }; - // list files to find min version - let version = async { - let mut min_version: i64 = version_start; - let prefix = Some(log_store.log_path()); - let offset_path = commit_uri_from_version(version_start); - let object_store = log_store.object_store(None); - - // Manually filter until we can provide direction in https://github.com/apache/arrow-rs/issues/6274 - let mut files = object_store - .list(prefix) - .try_filter(move |f| futures::future::ready(f.location < offset_path)) - .boxed(); - - while let Some(obj_meta) = files.next().await { - let obj_meta = obj_meta?; - if let Some(log_version) = extract_version_from_filename(obj_meta.location.as_ref()) { - min_version = min(min_version, log_version); - } - } - - if min_version < 0 { - return Err(DeltaTableError::not_a_table(log_store.root_uri())); + let storage = log_store.engine(None).await.storage_handler(); + let log_root = log_store.log_root_url(); + + let segment = spawn_blocking(move || { + LogSegment::for_table_changes(storage.as_ref(), log_root, current_version as u64, None) + }) + .await + .map_err(|e| DeltaTableError::Generic(e.to_string()))? + .map_err(|e| { + if e.to_string() + .contains(&format!("to have version {}", current_version)) + { + DeltaTableError::InvalidVersion(current_version) + } else { + DeltaTableError::Generic(e.to_string()) } + })?; - Ok::(min_version) - } - .await?; - Ok(version) + Ok(segment.end_version as i64) } /// Read delta log for a specific version @@ -586,37 +731,53 @@ pub async fn abort_commit_entry( } #[cfg(test)] -mod tests { +pub(crate) mod tests { + use futures::TryStreamExt; + use super::*; #[test] fn logstore_with_invalid_url() { let location = Url::parse("nonexistent://table").unwrap(); - let store = logstore_for(location, HashMap::default(), None); + + let store = logstore_for(location, StorageConfig::default()); assert!(store.is_err()); } #[test] fn logstore_with_memory() { - let location = Url::parse("memory://table").unwrap(); - let store = logstore_for(location, HashMap::default(), None); + let location = Url::parse("memory:///table").unwrap(); + let store = logstore_for(location, StorageConfig::default()); assert!(store.is_ok()); } #[test] fn logstore_with_memory_and_rt() { - let location = Url::parse("memory://table").unwrap(); - let store = logstore_for(location, HashMap::default(), Some(IORuntime::default())); + let location = Url::parse("memory:///table").unwrap(); + let store = logstore_for( + location, + StorageConfig::default().with_io_runtime(IORuntime::default()), + ); assert!(store.is_ok()); } + #[test] + fn test_logstore_ext() { + let location = Url::parse("memory:///table").unwrap(); + let store = logstore_for(location, StorageConfig::default()).unwrap(); + let table_url = store.table_root_url(); + assert!(table_url.path().ends_with('/')); + let log_url = store.log_root_url(); + assert!(log_url.path().ends_with("_delta_log/")); + } + #[tokio::test] async fn test_is_location_a_table() { use object_store::path::Path; use object_store::{PutOptions, PutPayload}; - let location = Url::parse("memory://table").unwrap(); + let location = Url::parse("memory:///table").unwrap(); let store = - logstore_for(location, HashMap::default(), None).expect("Failed to get logstore"); + logstore_for(location, StorageConfig::default()).expect("Failed to get logstore"); assert!(!store .is_delta_table_location() .await @@ -644,9 +805,9 @@ mod tests { async fn test_is_location_a_table_commit() { use object_store::path::Path; use object_store::{PutOptions, PutPayload}; - let location = Url::parse("memory://table").unwrap(); + let location = Url::parse("memory:///table").unwrap(); let store = - logstore_for(location, HashMap::default(), None).expect("Failed to get logstore"); + logstore_for(location, StorageConfig::default()).expect("Failed to get logstore"); assert!(!store .is_delta_table_location() .await @@ -657,7 +818,7 @@ mod tests { let _put = store .object_store(None) .put_opts( - &Path::from("_delta_log/0.json"), + &Path::from("_delta_log/00000000000000000000.json"), payload, PutOptions::default(), ) @@ -674,9 +835,9 @@ mod tests { async fn test_is_location_a_table_checkpoint() { use object_store::path::Path; use object_store::{PutOptions, PutPayload}; - let location = Url::parse("memory://table").unwrap(); + let location = Url::parse("memory:///table").unwrap(); let store = - logstore_for(location, HashMap::default(), None).expect("Failed to get logstore"); + logstore_for(location, StorageConfig::default()).expect("Failed to get logstore"); assert!(!store .is_delta_table_location() .await @@ -687,7 +848,7 @@ mod tests { let _put = store .object_store(None) .put_opts( - &Path::from("_delta_log/0.checkpoint.parquet"), + &Path::from("_delta_log/00000000000000000000.checkpoint.parquet"), payload, PutOptions::default(), ) @@ -704,9 +865,9 @@ mod tests { async fn test_is_location_a_table_crc() { use object_store::path::Path; use object_store::{PutOptions, PutPayload}; - let location = Url::parse("memory://table").unwrap(); + let location = Url::parse("memory:///table").unwrap(); let store = - logstore_for(location, HashMap::default(), None).expect("Failed to get logstore"); + logstore_for(location, StorageConfig::default()).expect("Failed to get logstore"); assert!(!store .is_delta_table_location() .await @@ -718,7 +879,7 @@ mod tests { let _put = store .object_store(None) .put_opts( - &Path::from("_delta_log/.0.crc.crc"), + &Path::from("_delta_log/.00000000000000000000.crc.crc"), payload.clone(), PutOptions::default(), ) @@ -728,7 +889,7 @@ mod tests { let _put = store .object_store(None) .put_opts( - &Path::from("_delta_log/.0.json.crc"), + &Path::from("_delta_log/.00000000000000000000.json.crc"), payload.clone(), PutOptions::default(), ) @@ -738,7 +899,7 @@ mod tests { let _put = store .object_store(None) .put_opts( - &Path::from("_delta_log/0.crc"), + &Path::from("_delta_log/00000000000000000000.crc"), payload.clone(), PutOptions::default(), ) @@ -749,7 +910,7 @@ mod tests { let _put = store .object_store(None) .put_opts( - &Path::from("_delta_log/0.json"), + &Path::from("_delta_log/00000000000000000000.json"), payload.clone(), PutOptions::default(), ) @@ -766,9 +927,9 @@ mod tests { /// :w #[tokio::test] async fn test_peek_with_invalid_json() -> DeltaResult<()> { - use crate::storage::object_store::memory::InMemory; + use crate::logstore::object_store::memory::InMemory; let memory_store = Arc::new(InMemory::new()); - let log_path = Path::from("_delta_log/00000000000000000001.json"); + let log_path = Path::from("delta-table/_delta_log/00000000000000000001.json"); let log_content = r#"{invalid_json"#; @@ -788,10 +949,21 @@ mod tests { assert!(result.is_err()); Ok(()) } + + /// Collect list stream + pub async fn flatten_list_stream( + storage: &object_store::DynObjectStore, + prefix: Option<&Path>, + ) -> object_store::Result> { + storage + .list(prefix) + .map_ok(|meta| meta.location) + .try_collect::>() + .await + } } -#[cfg(feature = "datafusion")] -#[cfg(test)] +#[cfg(all(test, feature = "datafusion"))] mod datafusion_tests { use super::*; use url::Url; diff --git a/crates/core/src/logstore/storage/mod.rs b/crates/core/src/logstore/storage/mod.rs new file mode 100644 index 0000000000..97508e4274 --- /dev/null +++ b/crates/core/src/logstore/storage/mod.rs @@ -0,0 +1,145 @@ +//! Object storage backend abstraction layer for Delta Table transaction logs and data +use std::sync::{Arc, LazyLock}; + +use dashmap::DashMap; +use object_store::path::Path; +use object_store::{DynObjectStore, ObjectStore}; +use url::Url; + +use crate::{DeltaResult, DeltaTableError}; +use deltalake_derive::DeltaConfig; + +pub use retry_ext::ObjectStoreRetryExt; +pub use runtime::{DeltaIOStorageBackend, IORuntime}; + +pub(super) mod retry_ext; +pub(super) mod runtime; +pub(super) mod utils; + +static DELTA_LOG_PATH: LazyLock = LazyLock::new(|| Path::from("_delta_log")); + +/// Sharable reference to [`ObjectStore`] +pub type ObjectStoreRef = Arc; + +pub trait ObjectStoreRegistry: Send + Sync + std::fmt::Debug + 'static + Clone { + /// If a store with the same key existed before, it is replaced and returned + fn register_store( + &self, + url: &Url, + store: Arc, + ) -> Option>; + + /// Get a suitable store for the provided URL. For example: + /// If no [`ObjectStore`] found for the `url`, ad-hoc discovery may be executed depending on + /// the `url` and [`ObjectStoreRegistry`] implementation. An [`ObjectStore`] may be lazily + /// created and registered. + fn get_store(&self, url: &Url) -> DeltaResult>; +} + +/// The default [`ObjectStoreRegistry`] +#[derive(Clone)] +pub struct DefaultObjectStoreRegistry { + /// A map from scheme to object store that serve list / read operations for the store + object_stores: DashMap>, +} + +impl Default for DefaultObjectStoreRegistry { + fn default() -> Self { + Self::new() + } +} + +impl DefaultObjectStoreRegistry { + pub fn new() -> Self { + let object_stores: DashMap> = DashMap::new(); + Self { object_stores } + } +} + +impl std::fmt::Debug for DefaultObjectStoreRegistry { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + f.debug_struct("DefaultObjectStoreRegistry") + .field( + "schemes", + &self + .object_stores + .iter() + .map(|o| o.key().clone()) + .collect::>(), + ) + .finish() + } +} + +impl ObjectStoreRegistry for DefaultObjectStoreRegistry { + fn register_store( + &self, + url: &Url, + store: Arc, + ) -> Option> { + self.object_stores.insert(url.to_string(), store) + } + + fn get_store(&self, url: &Url) -> DeltaResult> { + self.object_stores + .get(&url.to_string()) + .map(|o| Arc::clone(o.value())) + .ok_or_else(|| { + DeltaTableError::generic(format!("No suitable object store found for '{url}'.")) + }) + } +} + +#[derive(Debug, Clone, Default, DeltaConfig)] +pub struct LimitConfig { + #[delta(alias = "concurrency_limit", env = "OBJECT_STORE_CONCURRENCY_LIMIT")] + pub max_concurrency: Option, +} + +#[cfg(test)] +#[allow(drop_bounds, unused)] +mod tests { + use std::collections::HashMap; + use std::env; + + use rstest::*; + + use super::*; + use crate::logstore::config::TryUpdateKey; + use crate::test_utils::with_env; + + #[test] + fn test_limit_config() { + let mut config = LimitConfig::default(); + assert!(config.max_concurrency.is_none()); + + config.try_update_key("concurrency_limit", "10").unwrap(); + assert_eq!(config.max_concurrency, Some(10)); + + config.try_update_key("max_concurrency", "20").unwrap(); + assert_eq!(config.max_concurrency, Some(20)); + } + + #[rstest] + fn test_limit_config_env() { + let _env = with_env(vec![("OBJECT_STORE_CONCURRENCY_LIMIT", "100")]); + + let mut config = LimitConfig::default(); + assert!(config.max_concurrency.is_none()); + + config.load_from_environment().unwrap(); + assert_eq!(config.max_concurrency, Some(100)); + } + + #[rstest] + fn test_limit_config_env_error() { + let registry = DefaultObjectStoreRegistry::default(); + + // try get non-existent key + let url = Url::parse("not-registered://host").unwrap(); + let err = registry.get_store(&url).unwrap_err(); + assert!(err + .to_string() + .contains("No suitable object store found for 'not-registered://host'.")); + } +} diff --git a/crates/core/src/storage/retry_ext.rs b/crates/core/src/logstore/storage/retry_ext.rs similarity index 74% rename from crates/core/src/storage/retry_ext.rs rename to crates/core/src/logstore/storage/retry_ext.rs index 546e7b7ad6..298d7124f7 100644 --- a/crates/core/src/storage/retry_ext.rs +++ b/crates/core/src/logstore/storage/retry_ext.rs @@ -1,8 +1,13 @@ //! Retry extension for [`ObjectStore`] -use object_store::{path::Path, Error, ObjectStore, PutPayload, PutResult, Result}; +use ::object_store::path::Path; +use ::object_store::{Error, ObjectStore, PutPayload, PutResult, Result}; use tracing::log::*; +use crate::logstore::config; + +impl ObjectStoreRetryExt for T {} + /// Retry extension for [`ObjectStore`] /// /// Read-only operations are retried by [`ObjectStore`] internally. However, PUT/DELETE operations @@ -72,4 +77,26 @@ pub trait ObjectStoreRetryExt: ObjectStore { } } -impl ObjectStoreRetryExt for T {} +impl config::TryUpdateKey for object_store::RetryConfig { + fn try_update_key(&mut self, key: &str, v: &str) -> crate::DeltaResult> { + match key { + "max_retries" => self.max_retries = config::parse_usize(v)?, + "retry_timeout" => self.retry_timeout = config::parse_duration(v)?, + "init_backoff" | "backoff_config.init_backoff" | "backoff.init_backoff" => { + self.backoff.init_backoff = config::parse_duration(v)? + } + "max_backoff" | "backoff_config.max_backoff" | "backoff.max_backoff" => { + self.backoff.max_backoff = config::parse_duration(v)?; + } + "base" | "backoff_config.base" | "backoff.base" => { + self.backoff.base = config::parse_f64(v)?; + } + _ => return Ok(None), + } + Ok(Some(())) + } + + fn load_from_environment(&mut self) -> crate::DeltaResult<()> { + Ok(()) + } +} diff --git a/crates/core/src/logstore/storage/runtime.rs b/crates/core/src/logstore/storage/runtime.rs new file mode 100644 index 0000000000..087e637235 --- /dev/null +++ b/crates/core/src/logstore/storage/runtime.rs @@ -0,0 +1,318 @@ +use std::ops::Range; +use std::sync::OnceLock; + +use bytes::Bytes; +use deltalake_derive::DeltaConfig; +use futures::future::BoxFuture; +use futures::stream::BoxStream; +use futures::FutureExt; +use futures::TryFutureExt; +use object_store::path::Path; +use object_store::{ + Error as ObjectStoreError, GetOptions, GetResult, ListResult, ObjectMeta, ObjectStore, + PutOptions, PutPayload, PutResult, Result as ObjectStoreResult, +}; +use object_store::{MultipartUpload, PutMultipartOpts}; +use serde::{Deserialize, Serialize}; +use tokio::runtime::{Builder as RuntimeBuilder, Handle, Runtime}; + +/// Creates static IO Runtime with optional configuration +fn io_rt(config: Option<&RuntimeConfig>) -> &Runtime { + static IO_RT: OnceLock = OnceLock::new(); + IO_RT.get_or_init(|| { + let rt = match config { + Some(config) => { + let mut builder = if let Some(true) = config.multi_threaded { + RuntimeBuilder::new_multi_thread() + } else { + RuntimeBuilder::new_current_thread() + }; + + if let Some(threads) = config.worker_threads { + builder.worker_threads(threads); + } + + match (config.enable_io, config.enable_time) { + (Some(true), Some(true)) => { + builder.enable_all(); + } + (Some(false), Some(true)) => { + builder.enable_time(); + } + _ => (), + }; + + #[cfg(unix)] + { + if let (Some(true), Some(false)) = (config.enable_io, config.enable_time) { + builder.enable_io(); + } + } + builder + .thread_name( + config + .thread_name + .clone() + .unwrap_or("IO-runtime".to_string()), + ) + .build() + } + _ => Runtime::new(), + }; + rt.expect("Failed to create a tokio runtime for IO.") + }) +} + +/// Configuration for Tokio runtime +#[derive(Debug, Clone, Serialize, Deserialize, Default, DeltaConfig)] +pub struct RuntimeConfig { + /// Whether to use a multi-threaded runtime + pub(crate) multi_threaded: Option, + /// Number of worker threads to use + pub(crate) worker_threads: Option, + /// Name of the thread + pub(crate) thread_name: Option, + /// Whether to enable IO + pub(crate) enable_io: Option, + /// Whether to enable time + pub(crate) enable_time: Option, +} + +/// Provide custom Tokio RT or a runtime config +#[derive(Debug, Clone)] +pub enum IORuntime { + /// Tokio RT handle + RT(Handle), + /// Configuration for tokio runtime + Config(RuntimeConfig), +} + +impl Default for IORuntime { + fn default() -> Self { + IORuntime::RT(io_rt(None).handle().clone()) + } +} + +impl IORuntime { + /// Retrieves the Tokio runtime for IO bound operations + pub fn get_handle(&self) -> Handle { + match self { + IORuntime::RT(handle) => handle, + IORuntime::Config(config) => io_rt(Some(config)).handle(), + } + .clone() + } +} + +/// Wraps any object store and runs IO in it's own runtime [EXPERIMENTAL] +#[derive(Clone)] +pub struct DeltaIOStorageBackend { + pub inner: T, + pub rt_handle: Handle, +} + +impl DeltaIOStorageBackend +where + T: ObjectStore + Clone, +{ + pub fn new(store: T, handle: Handle) -> Self { + Self { + inner: store, + rt_handle: handle, + } + } +} + +impl DeltaIOStorageBackend { + /// spawn tasks on IO runtime + pub fn spawn_io_rt( + &self, + f: F, + store: &T, + path: Path, + ) -> BoxFuture<'_, ObjectStoreResult> + where + F: for<'a> FnOnce(&'a T, &'a Path) -> BoxFuture<'a, ObjectStoreResult> + Send + 'static, + O: Send + 'static, + { + let store = store.clone(); + let fut = self.rt_handle.spawn(async move { f(&store, &path).await }); + fut.unwrap_or_else(|e| match e.try_into_panic() { + Ok(p) => std::panic::resume_unwind(p), + Err(e) => Err(ObjectStoreError::JoinError { source: e }), + }) + .boxed() + } + + /// spawn tasks on IO runtime + pub fn spawn_io_rt_from_to( + &self, + f: F, + store: &T, + from: Path, + to: Path, + ) -> BoxFuture<'_, ObjectStoreResult> + where + F: for<'a> FnOnce(&'a T, &'a Path, &'a Path) -> BoxFuture<'a, ObjectStoreResult> + + Send + + 'static, + O: Send + 'static, + { + let store = store.clone(); + let fut = self + .rt_handle + .spawn(async move { f(&store, &from, &to).await }); + fut.unwrap_or_else(|e| match e.try_into_panic() { + Ok(p) => std::panic::resume_unwind(p), + Err(e) => Err(ObjectStoreError::JoinError { source: e }), + }) + .boxed() + } +} + +impl std::fmt::Debug for DeltaIOStorageBackend { + fn fmt(&self, fmt: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { + write!(fmt, "DeltaIOStorageBackend({:?})", self.inner) + } +} + +impl std::fmt::Display for DeltaIOStorageBackend { + fn fmt(&self, fmt: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { + write!(fmt, "DeltaIOStorageBackend({})", self.inner) + } +} + +#[async_trait::async_trait] +impl ObjectStore for DeltaIOStorageBackend { + async fn put(&self, location: &Path, bytes: PutPayload) -> ObjectStoreResult { + self.spawn_io_rt( + |store, path| store.put(path, bytes), + &self.inner, + location.clone(), + ) + .await + } + + async fn put_opts( + &self, + location: &Path, + bytes: PutPayload, + options: PutOptions, + ) -> ObjectStoreResult { + self.spawn_io_rt( + |store, path| store.put_opts(path, bytes, options), + &self.inner, + location.clone(), + ) + .await + } + + async fn get(&self, location: &Path) -> ObjectStoreResult { + self.spawn_io_rt(|store, path| store.get(path), &self.inner, location.clone()) + .await + } + + async fn get_opts(&self, location: &Path, options: GetOptions) -> ObjectStoreResult { + self.spawn_io_rt( + |store, path| store.get_opts(path, options), + &self.inner, + location.clone(), + ) + .await + } + + async fn get_range(&self, location: &Path, range: Range) -> ObjectStoreResult { + self.spawn_io_rt( + |store, path| store.get_range(path, range), + &self.inner, + location.clone(), + ) + .await + } + + async fn head(&self, location: &Path) -> ObjectStoreResult { + self.spawn_io_rt( + |store, path| store.head(path), + &self.inner, + location.clone(), + ) + .await + } + + async fn delete(&self, location: &Path) -> ObjectStoreResult<()> { + self.spawn_io_rt( + |store, path| store.delete(path), + &self.inner, + location.clone(), + ) + .await + } + + fn list(&self, prefix: Option<&Path>) -> BoxStream<'static, ObjectStoreResult> { + self.inner.list(prefix) + } + + fn list_with_offset( + &self, + prefix: Option<&Path>, + offset: &Path, + ) -> BoxStream<'static, ObjectStoreResult> { + self.inner.list_with_offset(prefix, offset) + } + + async fn list_with_delimiter(&self, prefix: Option<&Path>) -> ObjectStoreResult { + self.inner.list_with_delimiter(prefix).await + } + + async fn copy(&self, from: &Path, to: &Path) -> ObjectStoreResult<()> { + self.spawn_io_rt_from_to( + |store, from_path, to_path| store.copy(from_path, to_path), + &self.inner, + from.clone(), + to.clone(), + ) + .await + } + + async fn copy_if_not_exists(&self, from: &Path, to: &Path) -> ObjectStoreResult<()> { + self.spawn_io_rt_from_to( + |store, from_path, to_path| store.copy_if_not_exists(from_path, to_path), + &self.inner, + from.clone(), + to.clone(), + ) + .await + } + + async fn rename_if_not_exists(&self, from: &Path, to: &Path) -> ObjectStoreResult<()> { + self.spawn_io_rt_from_to( + |store, from_path, to_path| store.rename_if_not_exists(from_path, to_path), + &self.inner, + from.clone(), + to.clone(), + ) + .await + } + + async fn put_multipart(&self, location: &Path) -> ObjectStoreResult> { + self.spawn_io_rt( + |store, path| store.put_multipart(path), + &self.inner, + location.clone(), + ) + .await + } + + async fn put_multipart_opts( + &self, + location: &Path, + options: PutMultipartOpts, + ) -> ObjectStoreResult> { + self.spawn_io_rt( + |store, path| store.put_multipart_opts(path, options), + &self.inner, + location.clone(), + ) + .await + } +} diff --git a/crates/core/src/storage/utils.rs b/crates/core/src/logstore/storage/utils.rs similarity index 77% rename from crates/core/src/storage/utils.rs rename to crates/core/src/logstore/storage/utils.rs index 7ea5464b31..55a2f1c977 100644 --- a/crates/core/src/storage/utils.rs +++ b/crates/core/src/logstore/storage/utils.rs @@ -1,23 +1,23 @@ //! Utility functions for working across Delta tables use chrono::DateTime; -use futures::TryStreamExt; use object_store::path::Path; -use object_store::{DynObjectStore, ObjectMeta, Result as ObjectStoreResult}; +use object_store::ObjectMeta; use crate::errors::{DeltaResult, DeltaTableError}; use crate::kernel::Add; -/// Collect list stream -pub async fn flatten_list_stream( - storage: &DynObjectStore, - prefix: Option<&Path>, -) -> ObjectStoreResult> { - storage - .list(prefix) - .map_ok(|meta| meta.location) - .try_collect::>() - .await +/// Return the uri of commit version. +/// +/// ```rust +/// # use deltalake_core::logstore::*; +/// use object_store::path::Path; +/// let uri = commit_uri_from_version(1); +/// assert_eq!(uri, Path::from("_delta_log/00000000000000000001.json")); +/// ``` +pub fn commit_uri_from_version(version: i64) -> Path { + let version = format!("{version:020}.json"); + super::DELTA_LOG_PATH.child(version.as_str()) } impl TryFrom for ObjectMeta { @@ -33,17 +33,17 @@ impl TryFrom<&Add> for ObjectMeta { fn try_from(value: &Add) -> DeltaResult { let last_modified = DateTime::from_timestamp_millis(value.modification_time).ok_or( - DeltaTableError::from(crate::protocol::ProtocolError::InvalidField(format!( + DeltaTableError::MetadataError(format!( "invalid modification_time: {:?}", value.modification_time - ))), + )), )?; Ok(Self { // TODO this won't work for absolute paths, since Paths are always relative to store. location: Path::parse(value.path.as_str())?, last_modified, - size: value.size as usize, + size: value.size as u64, e_tag: None, version: None, }) @@ -68,7 +68,6 @@ mod tests { base_row_id: None, default_row_commit_version: None, deletion_vector: None, - stats_parsed: None, clustering_provider: None, }; diff --git a/crates/core/src/operations/add_column.rs b/crates/core/src/operations/add_column.rs index e6646edb9c..75063963ba 100644 --- a/crates/core/src/operations/add_column.rs +++ b/crates/core/src/operations/add_column.rs @@ -6,8 +6,8 @@ use delta_kernel::schema::StructType; use futures::future::BoxFuture; use itertools::Itertools; -use super::transaction::{CommitBuilder, CommitProperties}; use super::{CustomExecuteHandler, Operation}; +use crate::kernel::transaction::{CommitBuilder, CommitProperties}; use crate::kernel::{StructField, StructTypeExt}; use crate::logstore::LogStoreRef; use crate::operations::cast::merge_schema::merge_delta_struct; diff --git a/crates/core/src/operations/add_feature.rs b/crates/core/src/operations/add_feature.rs index 31dbb928bf..7d84b31590 100644 --- a/crates/core/src/operations/add_feature.rs +++ b/crates/core/src/operations/add_feature.rs @@ -2,12 +2,12 @@ use std::sync::Arc; -use delta_kernel::table_features::{ReaderFeatures, WriterFeatures}; +use delta_kernel::table_features::{ReaderFeature, WriterFeature}; use futures::future::BoxFuture; use itertools::Itertools; -use super::transaction::{CommitBuilder, CommitProperties}; use super::{CustomExecuteHandler, Operation}; +use crate::kernel::transaction::{CommitBuilder, CommitProperties}; use crate::kernel::TableFeatures; use crate::logstore::LogStoreRef; use crate::protocol::DeltaOperation; @@ -102,8 +102,8 @@ impl std::future::IntoFuture for AddTableFeatureBuilder { this.pre_execute(operation_id).await?; let (reader_features, writer_features): ( - Vec>, - Vec>, + Vec>, + Vec>, ) = name.iter().map(|v| v.to_reader_writer_features()).unzip(); let reader_features = reader_features.into_iter().flatten().collect_vec(); let writer_features = writer_features.into_iter().flatten().collect_vec(); @@ -157,7 +157,7 @@ mod tests { writer::test_utils::{create_bare_table, get_record_batch}, DeltaOps, }; - use delta_kernel::table_features::{ReaderFeatures, WriterFeatures}; + use delta_kernel::table_features::{ReaderFeature, WriterFeature}; use delta_kernel::DeltaResult; #[tokio::test] @@ -181,7 +181,7 @@ mod tests { .unwrap() .writer_features .unwrap_or_default() - .contains(&WriterFeatures::ChangeDataFeed)); + .contains(&WriterFeature::ChangeDataFeed)); let result = DeltaOps(result) .add_feature() @@ -195,13 +195,13 @@ mod tests { .writer_features .clone() .unwrap_or_default() - .contains(&WriterFeatures::DeletionVectors)); + .contains(&WriterFeature::DeletionVectors)); assert!(¤t_protocol .reader_features .clone() .unwrap_or_default() - .contains(&ReaderFeatures::DeletionVectors)); - assert_eq!(result.version(), 2); + .contains(&ReaderFeature::DeletionVectors)); + assert_eq!(result.version(), Some(2)); Ok(()) } diff --git a/crates/core/src/operations/cast/mod.rs b/crates/core/src/operations/cast/mod.rs index cf4f39262b..86d9f3f482 100644 --- a/crates/core/src/operations/cast/mod.rs +++ b/crates/core/src/operations/cast/mod.rs @@ -178,11 +178,14 @@ pub fn cast_record_batch( ..Default::default() }; - let s = StructArray::new( - batch.schema().as_ref().to_owned().fields, - batch.columns().to_owned(), - None, - ); + // Can be simplified with StructArray::try_new_with_length in arrow 55.1 + let col_arrays = batch.columns().to_owned(); + let s = if col_arrays.is_empty() { + StructArray::new_empty_fields(batch.num_rows(), None) + } else { + StructArray::new(batch.schema().as_ref().to_owned().fields, col_arrays, None) + }; + let struct_array = cast_struct(&s, target_schema.fields(), &cast_options, add_missing)?; Ok(RecordBatch::try_new_with_options( @@ -206,6 +209,7 @@ mod tests { }; use arrow::buffer::{Buffer, NullBuffer}; use arrow_schema::{DataType, Field, FieldRef, Fields, Schema, SchemaRef}; + use delta_kernel::engine::arrow_conversion::TryIntoKernel as _; use delta_kernel::schema::MetadataValue; use itertools::Itertools; @@ -230,7 +234,7 @@ mod tests { let result = merge_arrow_schema(left_schema, right_schema, true).unwrap(); assert_eq!(result.fields().len(), 1); - let delta_type: DeltaDataType = result.fields()[0].data_type().try_into().unwrap(); + let delta_type: DeltaDataType = result.fields()[0].data_type().try_into_kernel().unwrap(); assert_eq!(delta_type, DeltaDataType::STRING); assert!(result.fields()[0].is_nullable()); } @@ -280,7 +284,7 @@ mod tests { let result = merge_arrow_schema(left_schema, right_schema, true).unwrap(); assert_eq!(result.fields().len(), 1); - let delta_type: DeltaDataType = result.fields()[0].data_type().try_into().unwrap(); + let delta_type: DeltaDataType = result.fields()[0].data_type().try_into_kernel().unwrap(); assert_eq!( delta_type, DeltaDataType::Array(Box::new(DeltaArrayType::new(DeltaDataType::STRING, false))) diff --git a/crates/core/src/operations/cdc.rs b/crates/core/src/operations/cdc.rs index 5e950402b8..4d985dc216 100644 --- a/crates/core/src/operations/cdc.rs +++ b/crates/core/src/operations/cdc.rs @@ -70,7 +70,7 @@ pub(crate) fn should_write_cdc(snapshot: &DeltaTableState) -> DeltaResult // the Option> can get filled with an empty set, checking for the value // explicitly if snapshot.protocol().min_writer_version == 7 - && !features.contains(&delta_kernel::table_features::WriterFeatures::ChangeDataFeed) + && !features.contains(&delta_kernel::table_features::WriterFeature::ChangeDataFeed) { // If the writer feature has not been set, then the table should not have CDC written // to it. Otherwise fallback to the configured table configuration @@ -95,7 +95,7 @@ mod tests { use arrow_schema::Schema; use datafusion::assert_batches_sorted_eq; use datafusion::datasource::{MemTable, TableProvider}; - use delta_kernel::table_features::WriterFeatures; + use delta_kernel::table_features::WriterFeature; /// A simple test which validates primitive writer version 1 tables should /// not write Change Data Files @@ -175,7 +175,7 @@ mod tests { #[tokio::test] async fn test_should_write_cdc_v7_table_with_writer_feature() { let protocol = - Protocol::new(1, 7).append_writer_features(vec![WriterFeatures::ChangeDataFeed]); + Protocol::new(1, 7).append_writer_features(vec![WriterFeature::ChangeDataFeed]); let actions = vec![Action::Protocol(protocol)]; let mut table: DeltaTable = DeltaOps::new_in_memory() .create() diff --git a/crates/core/src/operations/constraints.rs b/crates/core/src/operations/constraints.rs index 7be5205574..287a32c9b3 100644 --- a/crates/core/src/operations/constraints.rs +++ b/crates/core/src/operations/constraints.rs @@ -7,13 +7,17 @@ use datafusion::execution::{SendableRecordBatchStream, TaskContext}; use datafusion::prelude::SessionContext; use datafusion_common::ToDFSchema; use datafusion_physical_plan::ExecutionPlan; +use delta_kernel::table_features::WriterFeature; use futures::future::BoxFuture; use futures::StreamExt; +use super::datafusion_utils::into_expr; +use super::{CustomExecuteHandler, Operation}; use crate::delta_datafusion::expr::fmt_expr_to_sql; use crate::delta_datafusion::{ register_store, DeltaDataChecker, DeltaScanBuilder, DeltaSessionContext, }; +use crate::kernel::transaction::{CommitBuilder, CommitProperties}; use crate::kernel::Protocol; use crate::logstore::LogStoreRef; use crate::operations::datafusion_utils::Expression; @@ -21,11 +25,6 @@ use crate::protocol::DeltaOperation; use crate::table::state::DeltaTableState; use crate::table::Constraint; use crate::{DeltaResult, DeltaTable, DeltaTableError}; -use delta_kernel::table_features::WriterFeatures; - -use super::datafusion_utils::into_expr; -use super::transaction::{CommitBuilder, CommitProperties}; -use super::{CustomExecuteHandler, Operation}; /// Build a constraint to add to a table pub struct ConstraintBuilder { @@ -201,7 +200,7 @@ impl std::future::IntoFuture for ConstraintBuilder { } else { let current_features = old_protocol.writer_features.clone(); if let Some(mut features) = current_features { - features.insert(WriterFeatures::CheckConstraints); + features.insert(WriterFeature::CheckConstraints); Some(features) } else { current_features @@ -327,7 +326,7 @@ mod tests { .with_constraint("id", "value < 1000") .await?; let version = table.version(); - assert_eq!(version, 1); + assert_eq!(version, Some(1)); let expected_expr = "value < 1000"; assert_eq!(get_constraint_op_params(&mut table).await, expected_expr); @@ -352,7 +351,7 @@ mod tests { .with_constraint("valid_values", col("value").lt(lit(1000))) .await?; let version = table.version(); - assert_eq!(version, 1); + assert_eq!(version, Some(1)); let expected_expr = "value < 1000"; assert_eq!(get_constraint_op_params(&mut table).await, expected_expr); @@ -368,7 +367,7 @@ mod tests { async fn test_constraint_case_sensitive() -> DeltaResult<()> { let arrow_schema = Arc::new(ArrowSchema::new(vec![ Field::new("Id", ArrowDataType::Utf8, true), - Field::new("vAlue", ArrowDataType::Int32, true), + Field::new("vAlue", ArrowDataType::Int32, true), // spellchecker:disable-line Field::new("mOdifieD", ArrowDataType::Utf8, true), ])); @@ -390,12 +389,12 @@ mod tests { let mut table = DeltaOps(table) .add_constraint() - .with_constraint("valid_values", "vAlue < 1000") + .with_constraint("valid_values", "vAlue < 1000") // spellchecker:disable-line .await?; let version = table.version(); - assert_eq!(version, 1); + assert_eq!(version, Some(1)); - let expected_expr = "vAlue < 1000"; + let expected_expr = "\"vAlue\" < 1000"; // spellchecker:disable-line assert_eq!(get_constraint_op_params(&mut table).await, expected_expr); assert_eq!( get_constraint(&table, "delta.constraints.valid_values"), diff --git a/crates/core/src/operations/convert_to_delta.rs b/crates/core/src/operations/convert_to_delta.rs index f4c563fbc5..96f657efdb 100644 --- a/crates/core/src/operations/convert_to_delta.rs +++ b/crates/core/src/operations/convert_to_delta.rs @@ -6,6 +6,8 @@ use std::str::{FromStr, Utf8Error}; use std::sync::Arc; use arrow_schema::{ArrowError, Schema as ArrowSchema}; +use delta_kernel::engine::arrow_conversion::TryIntoKernel as _; +use delta_kernel::schema::StructType; use futures::future::{self, BoxFuture}; use futures::TryStreamExt; use indexmap::IndexMap; @@ -16,9 +18,12 @@ use percent_encoding::percent_decode_str; use tracing::debug; use uuid::Uuid; +use super::{CustomExecuteHandler, Operation}; +use crate::kernel::transaction::CommitProperties; +use crate::logstore::StorageConfig; use crate::operations::get_num_idx_cols_and_stats_columns; use crate::{ - kernel::{scalars::ScalarExt, Add, DataType, Schema, StructField}, + kernel::{scalars::ScalarExt, Add, DataType, StructField}, logstore::{LogStore, LogStoreRef}, operations::create::CreateBuilder, protocol::SaveMode, @@ -28,9 +33,6 @@ use crate::{ DeltaResult, DeltaTable, DeltaTableError, ObjectStoreError, NULL_PARTITION_VALUE_DATA_PATH, }; -use super::transaction::CommitProperties; -use super::{CustomExecuteHandler, Operation}; - /// Error converting a Parquet table to a Delta table #[derive(Debug, thiserror::Error)] enum Error { @@ -253,10 +255,12 @@ impl ConvertToDeltaBuilder { self.log_store = if let Some(log_store) = self.log_store { Some(log_store) } else if let Some(location) = self.location.clone() { + let storage_config = + StorageConfig::parse_options(self.storage_options.clone().unwrap_or_default())?; + Some(crate::logstore::logstore_for( ensure_table_uri(location)?, - self.storage_options.clone().unwrap_or_default(), - None, // TODO: allow runtime to be passed into builder + storage_config, )?) } else { return Err(Error::MissingLocation); @@ -352,11 +356,11 @@ impl ConvertToDeltaBuilder { subpath = iter.next(); } - let batch_builder = ParquetRecordBatchStreamBuilder::new(ParquetObjectReader::new( - object_store.clone(), - file.clone(), - )) - .await?; + let object_reader = + ParquetObjectReader::new(object_store.clone(), file.location.clone()) + .with_file_size(file.size); + + let batch_builder = ParquetRecordBatchStreamBuilder::new(object_reader).await?; // Fetch the stats let parquet_metadata = batch_builder.metadata(); @@ -412,7 +416,7 @@ impl ConvertToDeltaBuilder { // Merge parquet file schemas // This step is needed because timestamp will not be preserved when copying files in S3. We can't use the schema of the latest parqeut file as Delta table's schema - let schema = Schema::try_from(&ArrowSchema::try_merge(arrow_schemas)?)?; + let schema: StructType = (&ArrowSchema::try_merge(arrow_schemas)?).try_into_kernel()?; let mut schema_fields = schema.fields().collect_vec(); schema_fields.append(&mut partition_schema_fields.values().collect::>()); @@ -471,12 +475,8 @@ mod tests { use tempfile::tempdir; use super::*; - use crate::{ - kernel::{DataType, PrimitiveType}, - open_table, - storage::StorageOptions, - Path, - }; + use crate::kernel::{DataType, PrimitiveType}; + use crate::{open_table, Path}; fn schema_field(key: &str, primitive: PrimitiveType, nullable: bool) -> StructField { StructField::new(key.to_string(), DataType::Primitive(primitive), nullable) @@ -503,7 +503,7 @@ mod tests { fn log_store(path: impl Into) -> LogStoreRef { let path: String = path.into(); let location = ensure_table_uri(path).expect("Failed to get the URI from the path"); - crate::logstore::logstore_for(location, StorageOptions::default(), None) + crate::logstore::logstore_for(location, StorageConfig::default()) .expect("Failed to create an object store") } @@ -567,7 +567,7 @@ mod tests { ) { assert_eq!( table.version(), - expected_version, + Some(expected_version), "Testing location: {test_data_from:?}" ); diff --git a/crates/core/src/operations/create.rs b/crates/core/src/operations/create.rs index 2ad04a93cd..d39d634adf 100644 --- a/crates/core/src/operations/create.rs +++ b/crates/core/src/operations/create.rs @@ -10,11 +10,11 @@ use serde_json::Value; use tracing::log::*; use uuid::Uuid; -use super::transaction::{CommitBuilder, CommitProperties, TableReference, PROTOCOL}; use super::{CustomExecuteHandler, Operation}; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::kernel::transaction::{CommitBuilder, CommitProperties, TableReference, PROTOCOL}; use crate::kernel::{Action, DataType, Metadata, Protocol, StructField, StructType}; -use crate::logstore::{LogStore, LogStoreRef}; +use crate::logstore::LogStoreRef; use crate::protocol::{DeltaOperation, SaveMode}; use crate::table::builder::ensure_table_uri; use crate::table::config::TableProperty; @@ -144,12 +144,7 @@ impl CreateBuilder { if let Value::Number(n) = v { n.as_i64().map_or_else( || MetadataValue::String(v.to_string()), - |i| { - i32::try_from(i) - .ok() - .map(MetadataValue::Number) - .unwrap_or_else(|| MetadataValue::String(v.to_string())) - }, + MetadataValue::Number, ) } else { MetadataValue::String(v.to_string()) @@ -234,8 +229,8 @@ impl CreateBuilder { self } - /// Provide a [`LogStore`] instance, that points at table location - pub fn with_log_store(mut self, log_store: Arc) -> Self { + /// Provide a [`LogStore`] instance + pub fn with_log_store(mut self, log_store: LogStoreRef) -> Self { self.log_store = Some(log_store); self } @@ -421,7 +416,7 @@ mod tests { .with_save_mode(SaveMode::Ignore) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_schema().unwrap(), &table_schema) } @@ -441,7 +436,7 @@ mod tests { .with_save_mode(SaveMode::Ignore) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_schema().unwrap(), &table_schema) } @@ -458,18 +453,18 @@ mod tests { .with_columns(schema.fields().cloned()) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); } #[tokio::test] async fn test_create_table_metadata() { let schema = get_delta_schema(); let table = CreateBuilder::new() - .with_location("memory://") + .with_location("memory:///") .with_columns(schema.fields().cloned()) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!( table.protocol().unwrap().min_reader_version, PROTOCOL.default_reader_version() @@ -488,7 +483,7 @@ mod tests { reader_features: None, }; let table = CreateBuilder::new() - .with_location("memory://") + .with_location("memory:///") .with_columns(schema.fields().cloned()) .with_actions(vec![Action::Protocol(protocol)]) .await @@ -497,7 +492,7 @@ mod tests { assert_eq!(table.protocol().unwrap().min_writer_version, 0); let table = CreateBuilder::new() - .with_location("memory://") + .with_location("memory:///") .with_columns(schema.fields().cloned()) .with_configuration_property(TableProperty::AppendOnly, Some("true")) .await @@ -514,6 +509,7 @@ mod tests { assert_eq!(String::from("true"), append) } + #[cfg(feature = "datafusion")] #[tokio::test] async fn test_create_table_save_mode() { let tmp_dir = tempfile::tempdir().unwrap(); @@ -524,7 +520,7 @@ mod tests { .with_columns(schema.fields().cloned()) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let first_id = table.metadata().unwrap().id.clone(); let log_store = table.log_store; @@ -556,6 +552,7 @@ mod tests { assert_ne!(table.metadata().unwrap().id, first_id) } + #[cfg(feature = "datafusion")] #[tokio::test] async fn test_create_or_replace_existing_table() { let batch = get_record_batch(None, false); @@ -565,7 +562,7 @@ mod tests { .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_files_count(), 1); let mut table = DeltaOps(table) @@ -575,12 +572,13 @@ mod tests { .await .unwrap(); table.load().await.unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); // Checks if files got removed after overwrite assert_eq!(table.get_files_count(), 0); } #[tokio::test] + #[cfg(feature = "datafusion")] async fn test_create_or_replace_existing_table_partitioned() { let batch = get_record_batch(None, false); let schema = get_delta_schema(); @@ -589,7 +587,7 @@ mod tests { .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_files_count(), 1); let mut table = DeltaOps(table) @@ -600,7 +598,7 @@ mod tests { .await .unwrap(); table.load().await.unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); // Checks if files got removed after overwrite assert_eq!(table.get_files_count(), 0); } @@ -615,7 +613,7 @@ mod tests { // Fail to create table with unknown Delta key let table = CreateBuilder::new() - .with_location("memory://") + .with_location("memory:///") .with_columns(schema.fields().cloned()) .with_configuration(config.clone()) .await; @@ -623,7 +621,7 @@ mod tests { // Succeed in creating table with unknown Delta key since we set raise_if_key_not_exists to false let table = CreateBuilder::new() - .with_location("memory://") + .with_location("memory:///") .with_columns(schema.fields().cloned()) .with_raise_if_key_not_exists(false) .with_configuration(config) diff --git a/crates/core/src/operations/delete.rs b/crates/core/src/operations/delete.rs index c6d943e1c0..6b59b9a4b7 100644 --- a/crates/core/src/operations/delete.rs +++ b/crates/core/src/operations/delete.rs @@ -40,7 +40,6 @@ use serde::Serialize; use super::cdc::should_write_cdc; use super::datafusion_utils::Expression; -use super::transaction::{CommitBuilder, CommitProperties, PROTOCOL}; use super::Operation; use crate::delta_datafusion::expr::fmt_expr_to_sql; use crate::delta_datafusion::logical::MetricObserver; @@ -51,6 +50,7 @@ use crate::delta_datafusion::{ DeltaTableProvider, }; use crate::errors::DeltaResult; +use crate::kernel::transaction::{CommitBuilder, CommitProperties, PROTOCOL}; use crate::kernel::{Action, Add, Remove}; use crate::logstore::LogStoreRef; use crate::operations::write::execution::{write_execution_plan, write_execution_plan_cdc}; @@ -491,7 +491,7 @@ mod tests { .with_partition_columns(partitions.unwrap_or_default()) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); table } @@ -533,12 +533,12 @@ mod tests { .with_save_mode(SaveMode::Append) .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 1); let (table, metrics) = DeltaOps(table).delete().await.unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert_eq!(table.get_files_count(), 0); assert_eq!(metrics.num_added_files, 0); assert_eq!(metrics.num_removed_files, 1); @@ -555,7 +555,7 @@ mod tests { // Deletes with no changes to state must not commit let (table, metrics) = DeltaOps(table).delete().await.unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert_eq!(metrics.num_added_files, 0); assert_eq!(metrics.num_removed_files, 0); assert_eq!(metrics.num_deleted_rows, 0); @@ -592,7 +592,7 @@ mod tests { .with_save_mode(SaveMode::Append) .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 1); let batch = RecordBatch::try_new( @@ -616,7 +616,7 @@ mod tests { .with_save_mode(SaveMode::Append) .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert_eq!(table.get_files_count(), 2); let (table, metrics) = DeltaOps(table) @@ -624,7 +624,7 @@ mod tests { .with_predicate(col("value").eq(lit(1))) .await .unwrap(); - assert_eq!(table.version(), 3); + assert_eq!(table.version(), Some(3)); assert_eq!(table.get_files_count(), 2); assert_eq!(metrics.num_added_files, 1); @@ -772,7 +772,7 @@ mod tests { .with_save_mode(SaveMode::Append) .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 2); let (table, metrics) = DeltaOps(table) @@ -780,7 +780,7 @@ mod tests { .with_predicate(col("modified").eq(lit("2021-02-03"))) .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert_eq!(table.get_files_count(), 1); assert_eq!(metrics.num_added_files, 0); @@ -829,7 +829,7 @@ mod tests { .with_save_mode(SaveMode::Append) .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 3); let (table, metrics) = DeltaOps(table) @@ -841,7 +841,7 @@ mod tests { ) .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert_eq!(table.get_files_count(), 2); assert_eq!(metrics.num_added_files, 0); @@ -944,7 +944,7 @@ mod tests { .with_configuration_property(TableProperty::EnableChangeDataFeed, Some("true")) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let schema = Arc::new(Schema::new(vec![Field::new( "value", @@ -961,14 +961,14 @@ mod tests { .write(vec![batch]) .await .expect("Failed to write first batch"); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let (table, _metrics) = DeltaOps(table) .delete() .with_predicate(col("value").eq(lit(2))) .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); let ctx = SessionContext::new(); let table = DeltaOps(table) @@ -1021,7 +1021,7 @@ mod tests { .with_configuration_property(TableProperty::EnableChangeDataFeed, Some("true")) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let schema = Arc::new(Schema::new(vec![ Field::new("year", DataType::Utf8, true), @@ -1045,14 +1045,14 @@ mod tests { .write(vec![batch]) .await .expect("Failed to write first batch"); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let (table, _metrics) = DeltaOps(table) .delete() .with_predicate(col("value").eq(lit(2))) .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); let ctx = SessionContext::new(); let table = DeltaOps(table) diff --git a/crates/core/src/operations/drop_constraints.rs b/crates/core/src/operations/drop_constraints.rs index 6743c3415a..2c362913d6 100644 --- a/crates/core/src/operations/drop_constraints.rs +++ b/crates/core/src/operations/drop_constraints.rs @@ -4,8 +4,8 @@ use std::sync::Arc; use futures::future::BoxFuture; -use super::transaction::{CommitBuilder, CommitProperties}; use super::{CustomExecuteHandler, Operation}; +use crate::kernel::transaction::{CommitBuilder, CommitProperties}; use crate::kernel::Action; use crate::logstore::LogStoreRef; use crate::protocol::DeltaOperation; diff --git a/crates/core/src/operations/filesystem_check.rs b/crates/core/src/operations/filesystem_check.rs index 32591f2812..e6474b285d 100644 --- a/crates/core/src/operations/filesystem_check.rs +++ b/crates/core/src/operations/filesystem_check.rs @@ -20,16 +20,15 @@ use std::time::UNIX_EPOCH; use futures::future::BoxFuture; use futures::StreamExt; -pub use object_store::path::Path; use object_store::ObjectStore; use serde::{de::Error as DeError, Deserialize, Deserializer, Serialize, Serializer}; use url::{ParseError, Url}; use uuid::Uuid; -use super::transaction::{CommitBuilder, CommitProperties}; use super::CustomExecuteHandler; use super::Operation; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::kernel::transaction::{CommitBuilder, CommitProperties}; use crate::kernel::{Action, Add, Remove}; use crate::logstore::LogStoreRef; use crate::protocol::DeltaOperation; diff --git a/crates/core/src/operations/load.rs b/crates/core/src/operations/load.rs index cf7f0c93ca..c52b19f03b 100644 --- a/crates/core/src/operations/load.rs +++ b/crates/core/src/operations/load.rs @@ -6,10 +6,10 @@ use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::physical_plan::{ExecutionPlan, SendableRecordBatchStream}; use futures::future::BoxFuture; -use super::transaction::PROTOCOL; use super::CustomExecuteHandler; use crate::delta_datafusion::DataFusionMixins; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::kernel::transaction::PROTOCOL; use crate::logstore::LogStoreRef; use crate::table::state::DeltaTableState; use crate::DeltaTable; diff --git a/crates/core/src/operations/load_cdf.rs b/crates/core/src/operations/load_cdf.rs index 90d2b5d364..0628c2092c 100644 --- a/crates/core/src/operations/load_cdf.rs +++ b/crates/core/src/operations/load_cdf.rs @@ -16,7 +16,10 @@ use std::time::SystemTime; use arrow_array::RecordBatch; use arrow_schema::{ArrowError, Field, Schema}; use chrono::{DateTime, Utc}; -use datafusion::datasource::physical_plan::{FileScanConfig, FileSource, ParquetSource}; +use datafusion::datasource::memory::DataSourceExec; +use datafusion::datasource::physical_plan::{ + FileGroup, FileScanConfigBuilder, FileSource, ParquetSource, +}; use datafusion::execution::SessionState; use datafusion::prelude::SessionContext; use datafusion_common::config::TableParquetOptions; @@ -135,14 +138,22 @@ impl CdfLoadBuilder { } else { self.calculate_earliest_version().await? }; - let latest_version = self.log_store.get_latest_version(start).await?; // Start from 0 since if start > latest commit, the returned commit is not a valid commit - - let mut end = self.ending_version.unwrap_or(latest_version); let mut change_files: Vec> = vec![]; let mut add_files: Vec> = vec![]; let mut remove_files: Vec> = vec![]; + // Start from 0 since if start > latest commit, the returned commit is not a valid commit + let latest_version = match self.log_store.get_latest_version(start).await { + Ok(latest_version) => latest_version, + Err(DeltaTableError::InvalidVersion(_)) if self.allow_out_of_range => { + return Ok((change_files, add_files, remove_files)); + } + Err(e) => return Err(e), + }; + + let mut end = self.ending_version.unwrap_or(latest_version); + if end > latest_version { end = latest_version; } @@ -172,10 +183,10 @@ impl CdfLoadBuilder { .log_store .read_commit_entry(latest_version) .await? - .ok_or(DeltaTableError::InvalidVersion(latest_version)); + .ok_or(DeltaTableError::InvalidVersion(latest_version))?; let latest_version_actions: Vec = - get_actions(latest_version, latest_snapshot_bytes?).await?; + get_actions(latest_version, latest_snapshot_bytes).await?; let latest_version_commit = latest_version_actions .iter() .find(|a| matches!(a, Action::CommitInfo(_))); @@ -374,32 +385,43 @@ impl CdfLoadBuilder { parquet_source.with_predicate(Arc::clone(&cdc_file_schema), Arc::clone(filters)); } let parquet_source: Arc = Arc::new(parquet_source); - let cdc_scan: Arc = FileScanConfig::new( - self.log_store.object_store_url(), - Arc::clone(&cdc_file_schema), - Arc::clone(&parquet_source), - ) - .with_file_groups(cdc_file_groups.into_values().collect()) - .with_table_partition_cols(cdc_partition_cols) - .build(); - - let add_scan: Arc = FileScanConfig::new( - self.log_store.object_store_url(), - Arc::clone(&add_remove_file_schema), - Arc::clone(&parquet_source), - ) - .with_file_groups(add_file_groups.into_values().collect()) - .with_table_partition_cols(add_remove_partition_cols.clone()) - .build(); - - let remove_scan: Arc = FileScanConfig::new( - self.log_store.object_store_url(), - Arc::clone(&add_remove_file_schema), - parquet_source, - ) - .with_file_groups(remove_file_groups.into_values().collect()) - .with_table_partition_cols(add_remove_partition_cols) - .build(); + let cdc_scan: Arc = DataSourceExec::from_data_source( + FileScanConfigBuilder::new( + self.log_store.object_store_url(), + Arc::clone(&cdc_file_schema), + Arc::clone(&parquet_source), + ) + .with_file_groups(cdc_file_groups.into_values().map(FileGroup::from).collect()) + .with_table_partition_cols(cdc_partition_cols) + .build(), + ); + + let add_scan: Arc = DataSourceExec::from_data_source( + FileScanConfigBuilder::new( + self.log_store.object_store_url(), + Arc::clone(&add_remove_file_schema), + Arc::clone(&parquet_source), + ) + .with_file_groups(add_file_groups.into_values().map(FileGroup::from).collect()) + .with_table_partition_cols(add_remove_partition_cols.clone()) + .build(), + ); + + let remove_scan: Arc = DataSourceExec::from_data_source( + FileScanConfigBuilder::new( + self.log_store.object_store_url(), + Arc::clone(&add_remove_file_schema), + parquet_source, + ) + .with_file_groups( + remove_file_groups + .into_values() + .map(FileGroup::from) + .collect(), + ) + .with_table_partition_cols(add_remove_partition_cols) + .build(), + ); // The output batches are then unioned to create a single output. Coalesce partitions is only here for the time // being for development. I plan to parallelize the reads once the base idea is correct. @@ -461,6 +483,7 @@ pub(crate) mod tests { use chrono::NaiveDateTime; use datafusion::prelude::SessionContext; use datafusion_common::assert_batches_sorted_eq; + use delta_kernel::engine::arrow_conversion::TryIntoArrow as _; use itertools::Itertools; use crate::test_utils::TestSchemas; @@ -647,10 +670,10 @@ pub(crate) mod tests { .await; assert!(table.is_err()); - assert!(table - .unwrap_err() - .to_string() - .contains("Invalid version. Start version 5 is greater than end version 4")); + assert!(matches!( + table.unwrap_err(), + DeltaTableError::InvalidVersion(5) + )); Ok(()) } @@ -795,9 +818,9 @@ pub(crate) mod tests { .with_configuration_property(TableProperty::EnableChangeDataFeed, Some("true")) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); - let schema = Arc::new(Schema::try_from(delta_schema)?); + let schema: Arc = Arc::new(delta_schema.try_into_arrow()?); let batch = RecordBatch::try_new( Arc::clone(&schema), @@ -827,14 +850,14 @@ pub(crate) mod tests { .write(vec![batch]) .await .expect("Failed to write first batch"); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let table = DeltaOps(table) .write([second_batch]) .with_save_mode(crate::protocol::SaveMode::Overwrite) .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); let ctx = SessionContext::new(); let cdf_scan = DeltaOps(table.clone()) diff --git a/crates/core/src/operations/merge/barrier.rs b/crates/core/src/operations/merge/barrier.rs index 7a459dd3d7..fceb3390fa 100644 --- a/crates/core/src/operations/merge/barrier.rs +++ b/crates/core/src/operations/merge/barrier.rs @@ -125,7 +125,9 @@ impl ExecutionPlan for MergeBarrierExec { impl DisplayAs for MergeBarrierExec { fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result { match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { + DisplayFormatType::Default + | DisplayFormatType::Verbose + | DisplayFormatType::TreeRender => { write!(f, "MergeBarrier",)?; Ok(()) } @@ -421,15 +423,6 @@ impl UserDefinedLogicalNodeCore for MergeBarrier { write!(f, "MergeBarrier") } - fn from_template( - &self, - exprs: &[datafusion_expr::Expr], - inputs: &[datafusion_expr::LogicalPlan], - ) -> Self { - self.with_exprs_and_inputs(exprs.to_vec(), inputs.to_vec()) - .unwrap() - } - fn with_exprs_and_inputs( &self, exprs: Vec, @@ -668,8 +661,8 @@ mod tests { MergeBarrierExec::new(exec, Arc::new("__delta_rs_path".to_string()), repartition); let survivors = merge.survivors(); - let coalsece = CoalesceBatchesExec::new(Arc::new(merge), 100); - let mut stream = coalsece.execute(0, task_ctx).unwrap(); + let coalescence = CoalesceBatchesExec::new(Arc::new(merge), 100); + let mut stream = coalescence.execute(0, task_ctx).unwrap(); (vec![stream.next().await.unwrap().unwrap()], survivors) } diff --git a/crates/core/src/operations/merge/filter.rs b/crates/core/src/operations/merge/filter.rs index c810db92f3..b63b35310a 100644 --- a/crates/core/src/operations/merge/filter.rs +++ b/crates/core/src/operations/merge/filter.rs @@ -15,7 +15,7 @@ use itertools::Itertools; use crate::delta_datafusion::execute_plan_to_batch; use crate::table::state::DeltaTableState; -use crate::DeltaResult; +use crate::{DeltaResult, DeltaTableError}; #[derive(Debug)] enum ReferenceTableCheck { @@ -382,9 +382,9 @@ pub(crate) async fn try_construct_early_filter( .map(|placeholder| { let col = items.column_by_name(placeholder).unwrap(); let value = ScalarValue::try_from_array(col, i)?; - DeltaResult::Ok((placeholder.to_owned(), value)) + Ok((placeholder.clone(), value)) }) - .try_collect()?; + .try_collect::<_, _, DeltaTableError>()?; Ok(replace_placeholders(filter.clone(), &replacements)) }) .collect::>>()? @@ -422,7 +422,7 @@ mod tests { let schema = get_arrow_schema(&None); let table = setup_table(Some(vec!["id"])).await; - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_files_count(), 0); let ctx = SessionContext::new(); @@ -514,7 +514,7 @@ mod tests { let schema = get_arrow_schema(&None); let table = setup_table(Some(vec!["modified"])).await; - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_files_count(), 0); let ctx = SessionContext::new(); @@ -569,7 +569,7 @@ mod tests { let schema = get_arrow_schema(&None); let table = setup_table(Some(vec!["modified"])).await; - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_files_count(), 0); let ctx = SessionContext::new(); @@ -629,7 +629,7 @@ mod tests { let schema = get_arrow_schema(&None); let table = setup_table(Some(vec!["modified"])).await; - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_files_count(), 0); let ctx = SessionContext::new(); @@ -695,7 +695,7 @@ mod tests { let schema = get_arrow_schema(&None); let table = setup_table(Some(vec!["modified"])).await; - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_files_count(), 0); let ctx = SessionContext::new(); @@ -767,7 +767,7 @@ mod tests { let schema = get_arrow_schema(&None); let table = setup_table(Some(vec!["modified"])).await; - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_files_count(), 0); let ctx = SessionContext::new(); diff --git a/crates/core/src/operations/merge/mod.rs b/crates/core/src/operations/merge/mod.rs index 782b070856..c1278a03e4 100644 --- a/crates/core/src/operations/merge/mod.rs +++ b/crates/core/src/operations/merge/mod.rs @@ -57,6 +57,7 @@ use datafusion_expr::{ Extension, LogicalPlan, LogicalPlanBuilder, UserDefinedLogicalNode, UNNAMED_TABLE, }; +use delta_kernel::engine::arrow_conversion::{TryIntoArrow as _, TryIntoKernel as _}; use delta_kernel::schema::{ColumnMetadataKey, StructType}; use filter::try_construct_early_filter; use futures::future::BoxFuture; @@ -66,9 +67,7 @@ use tracing::log::*; use uuid::Uuid; use self::barrier::{MergeBarrier, MergeBarrierExec}; - use super::datafusion_utils::{into_expr, maybe_into_expr, Expression}; -use super::transaction::{CommitProperties, PROTOCOL}; use super::{CustomExecuteHandler, Operation}; use crate::delta_datafusion::expr::{fmt_expr_to_sql, parse_predicate_expression}; use crate::delta_datafusion::logical::MetricObserver; @@ -78,16 +77,15 @@ use crate::delta_datafusion::{ register_store, DataFusionMixins, DeltaColumn, DeltaScan, DeltaScanConfigBuilder, DeltaSessionConfig, DeltaTableProvider, }; - +use crate::kernel::transaction::{CommitBuilder, CommitProperties, PROTOCOL}; use crate::kernel::{Action, Metadata, StructTypeExt}; use crate::logstore::LogStoreRef; use crate::operations::cast::merge_schema::{merge_arrow_field, merge_arrow_schema}; use crate::operations::cdc::*; use crate::operations::merge::barrier::find_node; -use crate::operations::transaction::CommitBuilder; use crate::operations::write::execution::write_execution_plan_v2; use crate::operations::write::generated_columns::{ - add_generated_columns, add_missing_generated_columns, + able_to_gc, add_generated_columns, add_missing_generated_columns, }; use crate::operations::write::WriterStatsConfig; use crate::protocol::{DeltaOperation, MergePredicate}; @@ -725,7 +723,7 @@ impl ExtensionPlanner for MergeMetricExtensionPlanner { #[allow(clippy::too_many_arguments)] async fn execute( predicate: Expression, - source: DataFrame, + mut source: DataFrame, log_store: LogStoreRef, snapshot: DeltaTableState, _state: SessionState, @@ -782,13 +780,18 @@ async fn execute( None => TableReference::bare(UNNAMED_TABLE), }; - let generated_col_expressions = snapshot - .schema() - .get_generated_columns() - .unwrap_or_default(); + let mut generated_col_exp = None; + let mut missing_generated_col = None; + + if able_to_gc(&snapshot)? { + let generated_col_expressions = snapshot.schema().get_generated_columns()?; + let (source_with_gc, missing_generated_columns) = + add_missing_generated_columns(source, &generated_col_expressions)?; - let (source, missing_generated_columns) = - add_missing_generated_columns(source, &generated_col_expressions)?; + source = source_with_gc; + generated_col_exp = Some(generated_col_expressions); + missing_generated_col = Some(missing_generated_columns); + } // This is only done to provide the source columns with a correct table reference. Just renaming the columns does not work let source = LogicalPlanBuilder::scan( source_name.clone(), @@ -961,7 +964,7 @@ async fn execute( )?; let schema = Arc::new(schema_builder.finish()); new_schema = Some(schema.clone()); - let schema_struct: StructType = schema.try_into()?; + let schema_struct: StructType = schema.try_into_kernel()?; if &schema_struct != snapshot.schema() { let action = Action::Metadata(Metadata::try_new( schema_struct, @@ -1085,7 +1088,7 @@ async fn execute( let mut write_projection_with_cdf = Vec::new(); let schema = if let Some(schema) = new_schema { - &schema.try_into()? + &schema.try_into_kernel()? } else { snapshot.schema() }; @@ -1109,7 +1112,7 @@ async fn execute( None => TableReference::none(), }; let name = delta_field.name(); - let mut cast_type: DataType = delta_field.data_type().try_into()?; + let mut cast_type: DataType = delta_field.data_type().try_into_arrow()?; // Receive the correct column reference given that some columns are only in source table let column = if let Some(field) = snapshot.schema().field(name) { @@ -1124,7 +1127,7 @@ async fn execute( } else { null_target_column = Some(cast( lit(ScalarValue::Null).alias(name), - delta_field.data_type().try_into()?, + delta_field.data_type().try_into_arrow()?, )); Column::new(source_qualifier.clone(), name) }; @@ -1348,12 +1351,16 @@ async fn execute( .select(write_projection)? }; - projected = add_generated_columns( - projected, - &generated_col_expressions, - &missing_generated_columns, - &state, - )?; + if let Some(generated_col_expressions) = generated_col_exp { + if let Some(missing_generated_columns) = missing_generated_col { + projected = add_generated_columns( + projected, + &generated_col_expressions, + &missing_generated_columns, + &state, + )?; + } + } let merge_final = &projected.into_unoptimized_plan(); let write = state.create_physical_plan(merge_final).await?; @@ -1595,6 +1602,7 @@ mod tests { use datafusion_expr::expr::Placeholder; use datafusion_expr::lit; use datafusion_expr::Expr; + use delta_kernel::engine::arrow_conversion::TryIntoKernel; use delta_kernel::schema::StructType; use itertools::Itertools; use regex::Regex; @@ -1613,7 +1621,7 @@ mod tests { .with_partition_columns(partitions.unwrap_or_default()) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); table } @@ -1714,14 +1722,14 @@ mod tests { let table = setup_table(None).await; let table = write_data(table, &schema).await; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 1); (table, merge_source(schema)) } async fn assert_merge(table: DeltaTable, metrics: MergeMetrics) { - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert!(table.get_files_count() >= 1); assert!(metrics.num_target_files_added >= 1); assert_eq!(metrics.num_target_files_removed, 1); @@ -2149,7 +2157,7 @@ mod tests { "+----+-------+------------+-------------+", ]; let actual = get_data(&table).await; - let expected_schema_struct: StructType = Arc::clone(&schema).try_into().unwrap(); + let expected_schema_struct: StructType = Arc::clone(&schema).try_into_kernel().unwrap(); assert_eq!(&expected_schema_struct, table.schema().unwrap()); assert_batches_sorted_eq!(&expected, &actual); } @@ -2217,7 +2225,7 @@ mod tests { "+----+-------+------------+-------------+", ]; let actual = get_data(&table).await; - let expected_schema_struct: StructType = Arc::clone(&schema).try_into().unwrap(); + let expected_schema_struct: StructType = Arc::clone(&schema).try_into_kernel().unwrap(); assert_eq!(&expected_schema_struct, table.schema().unwrap()); assert_batches_sorted_eq!(&expected, &actual); } @@ -2395,7 +2403,7 @@ mod tests { let table = setup_table(Some(vec!["modified"])).await; let table = write_data(table, &schema).await; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 2); let ctx = SessionContext::new(); @@ -2451,7 +2459,7 @@ mod tests { .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert!(table.get_files_count() >= 3); assert!(metrics.num_target_files_added >= 3); assert_eq!(metrics.num_target_files_removed, 2); @@ -2492,7 +2500,7 @@ mod tests { let schema = get_arrow_schema(&None); let table = setup_table(Some(vec!["modified"])).await; let table = write_data(table, &schema).await; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let ctx = SessionContext::new(); let batch = RecordBatch::try_new( Arc::clone(&schema), @@ -2531,7 +2539,7 @@ mod tests { .unwrap() .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); let commit_info = table.history(None).await.unwrap(); let last_commit = &commit_info[0]; let parameters = last_commit.operation_parameters.clone().unwrap(); @@ -2552,7 +2560,7 @@ mod tests { let table = setup_table(Some(vec!["id"])).await; let table = write_data(table, &schema).await; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 4); let ctx = SessionContext::new(); @@ -2591,7 +2599,7 @@ mod tests { .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert!(table.get_files_count() >= 3); assert_eq!(metrics.num_target_files_added, 3); assert_eq!(metrics.num_target_files_removed, 2); @@ -2631,7 +2639,7 @@ mod tests { let table = setup_table(Some(vec!["modified"])).await; let table = write_data(table, &schema).await; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 2); let ctx = SessionContext::new(); @@ -2695,7 +2703,7 @@ mod tests { .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert!(table.get_files_count() >= 3); assert!(metrics.num_target_files_added >= 3); assert_eq!(metrics.num_target_files_removed, 2); @@ -2741,7 +2749,7 @@ mod tests { let table = setup_table(Some(vec!["modified"])).await; let table = write_data(table, &schema).await; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 2); let ctx = SessionContext::new(); @@ -2769,7 +2777,7 @@ mod tests { .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert!(table.get_files_count() >= 2); assert_eq!(metrics.num_target_files_added, 2); assert_eq!(metrics.num_target_files_removed, 2); @@ -2811,7 +2819,7 @@ mod tests { let table = setup_table(Some(vec!["modified"])).await; let table = write_data(table, &schema).await; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 2); let ctx = SessionContext::new(); @@ -2839,7 +2847,7 @@ mod tests { .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert!(table.get_files_count() >= 2); assert_eq!(metrics.num_target_files_added, 1); assert_eq!(metrics.num_target_files_removed, 1); @@ -2880,7 +2888,7 @@ mod tests { let table = setup_table(Some(vec!["modified"])).await; let table = write_data(table, &schema).await; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 2); let ctx = SessionContext::new(); @@ -2908,7 +2916,7 @@ mod tests { .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert_eq!(table.get_files_count(), 2); assert_eq!(metrics.num_target_files_added, 2); assert_eq!(metrics.num_target_files_removed, 2); @@ -2944,7 +2952,7 @@ mod tests { let table = setup_table(Some(vec!["modified"])).await; let table = write_data(table, &schema).await; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 2); let ctx = SessionContext::new(); @@ -2974,7 +2982,7 @@ mod tests { .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert!(metrics.num_target_files_added == 1); assert_eq!(metrics.num_target_files_removed, 1); assert_eq!(metrics.num_target_rows_copied, 1); @@ -3014,7 +3022,7 @@ mod tests { let table = setup_table(Some(vec!["modified"])).await; let table = write_data(table, &schema).await; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 2); let ctx = SessionContext::new(); @@ -3043,7 +3051,7 @@ mod tests { .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert_eq!(table.get_files_count(), 2); assert_eq!(metrics.num_target_files_added, 2); assert_eq!(metrics.num_target_files_removed, 2); @@ -3079,7 +3087,7 @@ mod tests { let table = setup_table(Some(vec!["modified"])).await; let table = write_data(table, &schema).await; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 2); let ctx = SessionContext::new(); @@ -3109,7 +3117,7 @@ mod tests { .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert!(metrics.num_target_files_added == 1); assert_eq!(metrics.num_target_files_removed, 1); assert_eq!(metrics.num_target_rows_copied, 1); @@ -3146,7 +3154,7 @@ mod tests { let schema = get_arrow_schema(&None); let table = setup_table(Some(vec!["modified"])).await; - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_files_count(), 0); let ctx = SessionContext::new(); @@ -3190,7 +3198,7 @@ mod tests { .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert!(table.get_files_count() >= 2); assert!(metrics.num_target_files_added >= 2); assert_eq!(metrics.num_target_files_removed, 0); @@ -3233,7 +3241,7 @@ mod tests { ])); let table = setup_table(Some(vec!["modified"])).await; - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_files_count(), 0); let ctx = SessionContext::new(); @@ -3274,7 +3282,7 @@ mod tests { .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert!(table.get_files_count() >= 2); assert!(metrics.num_target_files_added >= 2); assert_eq!(metrics.num_target_files_removed, 0); @@ -3304,7 +3312,7 @@ mod tests { "+----+-------+-------------+------------+", ]; let actual = get_data(&table).await; - let expected_schema_struct: StructType = Arc::clone(&schema).try_into().unwrap(); + let expected_schema_struct: StructType = Arc::clone(&schema).try_into_kernel().unwrap(); assert_eq!(&expected_schema_struct, table.schema().unwrap()); assert_batches_sorted_eq!(&expected, &actual); } @@ -3318,7 +3326,7 @@ mod tests { true, ), StructField::new( - "vAlue".to_string(), + "vAlue".to_string(), // spellchecker:disable-line DataType::Primitive(PrimitiveType::Integer), true, ), @@ -3331,7 +3339,7 @@ mod tests { let arrow_schema = Arc::new(ArrowSchema::new(vec![ Field::new("Id", ArrowDataType::Utf8, true), - Field::new("vAlue", ArrowDataType::Int32, true), + Field::new("vAlue", ArrowDataType::Int32, true), // spellchecker:disable-line Field::new("mOdifieD", ArrowDataType::Utf8, true), ])); @@ -3358,7 +3366,7 @@ mod tests { let source = ctx.read_batch(batch).unwrap(); let table = write_data(table, &arrow_schema).await; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 1); let (table, _metrics) = DeltaOps(table) @@ -3368,7 +3376,7 @@ mod tests { .when_not_matched_insert(|insert| { insert .set("Id", "source.Id") - .set("vAlue", "source.vAlue + 1") + .set("vAlue", "source.vAlue + 1") // spellchecker:disable-line .set("mOdifieD", "source.mOdifieD") }) .unwrap() @@ -3377,7 +3385,7 @@ mod tests { let expected = vec![ "+----+-------+------------+", - "| Id | vAlue | mOdifieD |", + "| Id | vAlue | mOdifieD |", // spellchecker:disable-line "+----+-------+------------+", "| A | 1 | 2021-02-01 |", "| B | 10 | 2021-02-01 |", @@ -3654,7 +3662,7 @@ mod tests { .with_save_mode(SaveMode::Append) .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 1); let batch = RecordBatch::try_new( @@ -3769,7 +3777,7 @@ mod tests { .with_save_mode(SaveMode::Append) .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 1); let batch = RecordBatch::try_new( @@ -3879,7 +3887,7 @@ mod tests { .with_save_mode(SaveMode::Append) .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 1); let batch = RecordBatch::try_new( @@ -3958,7 +3966,7 @@ mod tests { assert_merge(table.clone(), metrics).await; // Just checking that the data wasn't actually written instead! - if let Ok(files) = crate::storage::utils::flatten_list_stream( + if let Ok(files) = crate::logstore::tests::flatten_list_stream( &table.object_store(), Some(&object_store::path::Path::from("_change_data")), ) @@ -3988,12 +3996,12 @@ mod tests { .with_configuration_property(TableProperty::EnableChangeDataFeed, Some("true")) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let schema = get_arrow_schema(&None); let table = write_data(table, &schema).await; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 1); let source = merge_source(schema); @@ -4081,7 +4089,7 @@ mod tests { .with_configuration_property(TableProperty::EnableChangeDataFeed, Some("true")) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let schema = get_arrow_schema(&None); @@ -4093,7 +4101,7 @@ mod tests { ])); let table = write_data(table, &schema).await; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 1); let source = merge_source(schema); let source = source.with_column("inserted_by", lit("new_value")).unwrap(); @@ -4138,7 +4146,7 @@ mod tests { "+----+-------+------------+-------------+", ]; let actual = get_data(&table).await; - let expected_schema_struct: StructType = source_schema.try_into().unwrap(); + let expected_schema_struct: StructType = source_schema.try_into_kernel().unwrap(); assert_eq!(&expected_schema_struct, table.schema().unwrap()); assert_batches_sorted_eq!(&expected, &actual); @@ -4198,12 +4206,12 @@ mod tests { .with_configuration_property(TableProperty::EnableChangeDataFeed, Some("true")) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let schema = get_arrow_schema(&None); let table = write_data(table, &schema).await; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 1); let source = merge_source(schema); diff --git a/crates/core/src/operations/mod.rs b/crates/core/src/operations/mod.rs index 9c95a5f0b7..d44268b8e9 100644 --- a/crates/core/src/operations/mod.rs +++ b/crates/core/src/operations/mod.rs @@ -25,6 +25,7 @@ use self::filesystem_check::FileSystemCheckBuilder; use self::optimize::OptimizeBuilder; use self::restore::RestoreBuilder; use self::set_tbl_properties::SetTablePropertiesBuilder; +use self::update_table_metadata::UpdateTableMetadataBuilder; use self::vacuum::VacuumBuilder; #[cfg(feature = "datafusion")] use self::{ @@ -45,11 +46,11 @@ pub mod create; pub mod drop_constraints; pub mod filesystem_check; pub mod restore; -pub mod transaction; pub mod update_field_metadata; +pub mod update_table_metadata; pub mod vacuum; -#[cfg(all(feature = "cdf", feature = "datafusion"))] +#[cfg(feature = "datafusion")] mod cdc; #[cfg(feature = "datafusion")] pub mod constraints; @@ -131,7 +132,7 @@ impl DeltaOps { /// use deltalake_core::DeltaOps; /// /// async { - /// let ops = DeltaOps::try_from_uri("memory://").await.unwrap(); + /// let ops = DeltaOps::try_from_uri("memory:///").await.unwrap(); /// }; /// ``` pub async fn try_from_uri(uri: impl AsRef) -> DeltaResult { @@ -172,7 +173,7 @@ impl DeltaOps { /// ``` #[must_use] pub fn new_in_memory() -> Self { - DeltaTableBuilder::from_uri("memory://") + DeltaTableBuilder::from_uri("memory:///") .build() .unwrap() .into() @@ -184,9 +185,9 @@ impl DeltaOps { /// use deltalake_core::DeltaOps; /// /// async { - /// let ops = DeltaOps::try_from_uri("memory://").await.unwrap(); + /// let ops = DeltaOps::try_from_uri("memory:///").await.unwrap(); /// let table = ops.create().with_table_name("my_table").await.unwrap(); - /// assert_eq!(table.version(), 0); + /// assert_eq!(table.version(), Some(0)); /// }; /// ``` #[must_use] @@ -304,6 +305,11 @@ impl DeltaOps { pub fn update_field_metadata(self) -> UpdateFieldMetadataBuilder { UpdateFieldMetadataBuilder::new(self.0.log_store, self.0.state.unwrap()) } + + /// Update table metadata + pub fn update_table_metadata(self) -> UpdateTableMetadataBuilder { + UpdateTableMetadataBuilder::new(self.0.log_store, self.0.state.unwrap()) + } } impl From for DeltaOps { diff --git a/crates/core/src/operations/optimize.rs b/crates/core/src/operations/optimize.rs index 503e09ffcb..2d108f2410 100644 --- a/crates/core/src/operations/optimize.rs +++ b/crates/core/src/operations/optimize.rs @@ -27,6 +27,7 @@ use std::time::{Duration, Instant, SystemTime, UNIX_EPOCH}; use arrow_array::RecordBatch; use arrow_schema::SchemaRef as ArrowSchemaRef; +use delta_kernel::engine::arrow_conversion::TryIntoArrow as _; use delta_kernel::expressions::Scalar; use futures::future::BoxFuture; use futures::stream::BoxStream; @@ -42,21 +43,18 @@ use serde::{de::Error as DeError, Deserialize, Deserializer, Serialize, Serializ use tracing::*; use uuid::Uuid; -use super::transaction::PROTOCOL; use super::write::writer::{PartitionWriter, PartitionWriterConfig}; use super::{CustomExecuteHandler, Operation}; +use crate::delta_datafusion::DeltaTableProvider; use crate::errors::{DeltaResult, DeltaTableError}; -use crate::kernel::Add; -use crate::kernel::{scalars::ScalarExt, Action, PartitionsExt, Remove}; -use crate::logstore::LogStoreRef; -use crate::operations::transaction::{CommitBuilder, CommitProperties, DEFAULT_RETRIES}; +use crate::kernel::transaction::{CommitBuilder, CommitProperties, DEFAULT_RETRIES, PROTOCOL}; +use crate::kernel::{scalars::ScalarExt, Action, Add, PartitionsExt, Remove}; +use crate::logstore::{LogStoreRef, ObjectStoreRef}; use crate::protocol::DeltaOperation; -use crate::storage::ObjectStoreRef; use crate::table::state::DeltaTableState; use crate::writer::utils::arrow_schema_without_partitions; use crate::{crate_version, DeltaTable, ObjectMeta, PartitionFilter}; -use crate::delta_datafusion::DeltaTableProvider; /// Metrics from Optimize #[derive(Default, Debug, PartialEq, Clone, Serialize, Deserialize)] #[serde(rename_all = "camelCase")] @@ -485,11 +483,7 @@ impl MergePlan { let mut partial_actions = files .iter() .map(|file_meta| { - create_remove( - file_meta.path.as_ref(), - &partition_values, - file_meta.size as i64, - ) + create_remove(file_meta.path.as_ref(), &partition_values, file_meta.size) }) .collect::, DeltaTableError>>()?; @@ -497,9 +491,9 @@ impl MergePlan { .iter() .fold(MetricDetails::default(), |mut curr, file| { curr.total_files += 1; - curr.total_size += file.size as i64; - curr.max = std::cmp::max(curr.max, file.size as i64); - curr.min = std::cmp::min(curr.min, file.size as i64); + curr.total_size += file.size; + curr.max = std::cmp::max(curr.max, file.size); + curr.min = std::cmp::min(curr.min, file.size); curr }); @@ -538,7 +532,7 @@ impl MergePlan { true, )?; partial_metrics.num_batches += 1; - writer.write(&batch).await.map_err(DeltaTableError::from)?; + writer.write(&batch).await?; } let add_actions = writer.close().await?.into_iter().map(|mut add| { @@ -629,11 +623,11 @@ impl MergePlan { let batch_stream = futures::stream::iter(files.clone()) .then(move |file| { let object_store_ref = object_store_ref.clone(); + let meta = ObjectMeta::try_from(file).unwrap(); async move { - let file_reader = ParquetObjectReader::new( - object_store_ref, - ObjectMeta::try_from(file).unwrap(), - ); + let file_reader = + ParquetObjectReader::new(object_store_ref, meta.location) + .with_file_size(meta.size); ParquetRecordBatchStreamBuilder::new(file_reader) .await? .build() @@ -670,7 +664,7 @@ impl MergePlan { let scan_config = DeltaScanConfigBuilder::default() .with_file_column(false) .with_schema(snapshot.input_schema()?) - .build(&snapshot)?; + .build(snapshot)?; // For each rewrite evaluate the predicate and then modify each expression // to either compute the new value or obtain the old one then write these batches @@ -811,8 +805,10 @@ pub fn create_merge_plan( target_size, predicate: serde_json::to_string(filters).ok(), }; - let file_schema = - arrow_schema_without_partitions(&Arc::new(snapshot.schema().try_into()?), partitions_keys); + let file_schema = arrow_schema_without_partitions( + &Arc::new(snapshot.schema().try_into_arrow()?), + partitions_keys, + ); Ok(MergePlan { operations, @@ -855,7 +851,7 @@ impl MergeBin { } fn add(&mut self, add: Add) { - self.size_bytes += add.size as i64; + self.size_bytes += add.size; self.files.push(add); } @@ -913,7 +909,7 @@ fn build_compaction_plan( 'files: for file in files { for bin in merge_bins.iter_mut() { - if bin.total_file_size() + file.size as i64 <= target_size { + if bin.total_file_size() + file.size <= target_size { bin.add(file); // Move to next file continue 'files; @@ -1050,7 +1046,8 @@ pub(super) mod zorder { use arrow_schema::DataType; use datafusion_common::DataFusionError; use datafusion_expr::{ - ColumnarValue, ScalarUDF, ScalarUDFImpl, Signature, TypeSignature, Volatility, + ColumnarValue, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, Signature, TypeSignature, + Volatility, }; use itertools::Itertools; use std::any::Any; @@ -1106,8 +1103,11 @@ pub(super) mod zorder { Ok(DataType::Binary) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { - zorder_key_datafusion(args) + fn invoke_with_args( + &self, + args: ScalarFunctionArgs, + ) -> datafusion_common::Result { + zorder_key_datafusion(&args.args) } } diff --git a/crates/core/src/operations/restore.rs b/crates/core/src/operations/restore.rs index af90851985..940dae71f7 100644 --- a/crates/core/src/operations/restore.rs +++ b/crates/core/src/operations/restore.rs @@ -33,15 +33,14 @@ use object_store::ObjectStore; use serde::Serialize; use uuid::Uuid; +use super::{CustomExecuteHandler, Operation}; +use crate::kernel::transaction::{CommitBuilder, CommitProperties, TransactionError}; use crate::kernel::{Action, Add, Protocol, Remove}; use crate::logstore::LogStoreRef; use crate::protocol::DeltaOperation; use crate::table::state::DeltaTableState; use crate::{DeltaResult, DeltaTable, DeltaTableConfig, DeltaTableError, ObjectStoreError}; -use super::transaction::{CommitBuilder, CommitProperties, TransactionError}; -use super::{CustomExecuteHandler, Operation}; - /// Errors that can occur during restore #[derive(thiserror::Error, Debug)] enum RestoreError { @@ -177,11 +176,15 @@ async fn execute( let version = match datetime_to_restore { Some(datetime) => { table.load_with_datetime(datetime).await?; - table.version() + table + .version() + .ok_or_else(|| DeltaTableError::NotInitialized)? } None => { table.load_version(version_to_restore.unwrap()).await?; - table.version() + table + .version() + .ok_or_else(|| DeltaTableError::NotInitialized)? } }; @@ -191,6 +194,9 @@ async fn execute( snapshot.version(), ))); } + + let metadata_restored_version = table.metadata()?; + let state_to_restore_files = table.snapshot()?.file_actions()?; let latest_state_files = snapshot.file_actions()?; let state_to_restore_files_set = @@ -279,6 +285,8 @@ async fn execute( actions.push(Action::Protocol(protocol)); actions.extend(files_to_add.into_iter().map(Action::Add)); actions.extend(files_to_remove.into_iter().map(Action::Remove)); + // Add the metadata from the restored version to undo e.g. constraint or field metadata changes + actions.push(Action::Metadata(metadata_restored_version.clone())); let operation = DeltaOperation::Restore { version: version_to_restore, @@ -361,3 +369,48 @@ impl std::future::IntoFuture for RestoreBuilder { }) } } + +#[cfg(test)] +mod tests { + + use crate::writer::test_utils::{create_bare_table, get_record_batch}; + use crate::{DeltaOps, DeltaResult}; + + /// Verify that restore respects constraints that were added/removed in previous version_to_restore + /// + #[tokio::test] + #[cfg(feature = "datafusion")] + async fn test_simple_restore_constraints() -> DeltaResult<()> { + let batch = get_record_batch(None, false); + let table = DeltaOps(create_bare_table()) + .write(vec![batch.clone()]) + .await?; + let first_v = table.version().unwrap(); + + let constraint = DeltaOps(table) + .add_constraint() + .with_constraint("my_custom_constraint", "value < 100") + .await; + let table = constraint.expect("Failed to add constraint to table"); + + let constraints = table + .state + .as_ref() + .unwrap() + .table_config() + .get_constraints(); + assert!(constraints.len() == 1); + assert_eq!(constraints[0].name, "my_custom_constraint"); + + let (table, _metrics) = DeltaOps(table) + .restore() + .with_version_to_restore(first_v) + .await?; + assert_ne!(table.version(), Some(first_v)); + + let constraints = table.state.unwrap().table_config().get_constraints(); + assert!(constraints.is_empty()); + + Ok(()) + } +} diff --git a/crates/core/src/operations/set_tbl_properties.rs b/crates/core/src/operations/set_tbl_properties.rs index 3e17f8c07c..ccbf120159 100644 --- a/crates/core/src/operations/set_tbl_properties.rs +++ b/crates/core/src/operations/set_tbl_properties.rs @@ -5,8 +5,8 @@ use std::sync::Arc; use futures::future::BoxFuture; -use super::transaction::{CommitBuilder, CommitProperties}; use super::{CustomExecuteHandler, Operation}; +use crate::kernel::transaction::{CommitBuilder, CommitProperties}; use crate::kernel::Action; use crate::logstore::LogStoreRef; use crate::protocol::DeltaOperation; diff --git a/crates/core/src/operations/update.rs b/crates/core/src/operations/update.rs index 7433ef57b2..40733ec182 100644 --- a/crates/core/src/operations/update.rs +++ b/crates/core/src/operations/update.rs @@ -45,32 +45,26 @@ use serde::Serialize; use tracing::log::*; use uuid::Uuid; -use super::{ - datafusion_utils::Expression, - transaction::{CommitBuilder, CommitProperties}, -}; -use super::{transaction::PROTOCOL, write::WriterStatsConfig}; +use super::datafusion_utils::Expression; +use super::write::WriterStatsConfig; use super::{ write::execution::{write_execution_plan, write_execution_plan_cdc}, CustomExecuteHandler, Operation, }; +use crate::delta_datafusion::{ + expr::fmt_expr_to_sql, + logical::MetricObserver, + physical::{find_metric_node, get_metric, MetricObserverExec}, + DataFusionMixins, DeltaColumn, DeltaScanConfigBuilder, DeltaSessionContext, DeltaTableProvider, +}; use crate::delta_datafusion::{find_files, planner::DeltaPlanner, register_store}; +use crate::kernel::transaction::{CommitBuilder, CommitProperties, PROTOCOL}; use crate::kernel::{Action, Remove}; use crate::logstore::LogStoreRef; use crate::operations::cdc::*; use crate::protocol::DeltaOperation; use crate::table::state::DeltaTableState; -use crate::{ - delta_datafusion::{ - expr::fmt_expr_to_sql, - logical::MetricObserver, - physical::{find_metric_node, get_metric, MetricObserverExec}, - DataFusionMixins, DeltaColumn, DeltaScanConfigBuilder, DeltaSessionContext, - DeltaTableProvider, - }, - DeltaTableError, -}; -use crate::{DeltaResult, DeltaTable}; +use crate::{DeltaResult, DeltaTable, DeltaTableError}; /// Custom column name used for marking internal [RecordBatch] rows as updated pub(crate) const UPDATE_PREDICATE_COLNAME: &str = "__delta_rs_update_predicate"; @@ -377,7 +371,9 @@ async fn execute( //let updated_df = df_with_predicate_and_metrics.clone(); // Disabling the select allows the coerce test to pass, still not sure why - let updated_df = df_with_predicate_and_metrics.select(expressions.clone())?; + let updated_df = df_with_predicate_and_metrics + .select(expressions.clone())? + .drop_columns(&[UPDATE_PREDICATE_COLNAME])?; let physical_plan = updated_df.clone().create_physical_plan().await?; let writer_stats_config = WriterStatsConfig::new( snapshot.table_config().num_indexed_cols(), @@ -387,7 +383,7 @@ async fn execute( .map(|v| v.iter().map(|v| v.to_string()).collect::>()), ); - let tracker = CDCTracker::new(df, updated_df.drop_columns(&[UPDATE_PREDICATE_COLNAME])?); + let tracker = CDCTracker::new(df, updated_df); let add_actions = write_execution_plan( Some(&snapshot), @@ -554,6 +550,7 @@ mod tests { use arrow_schema::DataType; use datafusion::assert_batches_sorted_eq; use datafusion::prelude::*; + use delta_kernel::engine::arrow_conversion::TryIntoArrow; use serde_json::json; use std::sync::Arc; @@ -566,7 +563,7 @@ mod tests { .with_partition_columns(partitions.unwrap_or_default()) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); table } @@ -605,6 +602,55 @@ mod tests { .expect_err("Remove action is included when Delta table is append-only. Should error"); } + // + #[tokio::test] + async fn test_update_predicate_left_in_data() -> DeltaResult<()> { + let schema = get_arrow_schema(&None); + let table = setup_table(None).await; + + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(arrow::array::StringArray::from(vec!["A", "B", "A", "A"])), + Arc::new(arrow::array::Int32Array::from(vec![1, 10, 10, 100])), + Arc::new(arrow::array::StringArray::from(vec![ + "2021-02-02", + "2021-02-02", + "2021-02-02", + "2021-02-02", + ])), + ], + )?; + + let table = write_batch(table, batch).await; + assert_eq!(table.version(), Some(1)); + + let (table, _) = DeltaOps(table) + .update() + .with_update("modified", lit("2023-05-14")) + .with_predicate(col("value").eq(lit(10))) + .await?; + + use parquet::arrow::async_reader::ParquetObjectReader; + use parquet::arrow::async_reader::ParquetRecordBatchStreamBuilder; + + for pq in table.get_files_iter()? { + let store = table.log_store().object_store(None); + let reader = ParquetObjectReader::new(store, pq); + let builder = ParquetRecordBatchStreamBuilder::new(reader).await?; + let schema = builder.schema(); + + assert!( + schema + .field_with_name("__delta_rs_update_predicate") + .is_err(), + "The schema contains __delta_rs_update_predicate which is incorrect!" + ); + assert_eq!(schema.fields.len(), 3, "Expected the Parquet file to only have three fields in the schema, something is amiss!"); + } + Ok(()) + } + #[tokio::test] async fn test_update_no_predicate() { let schema = get_arrow_schema(&None); @@ -626,7 +672,7 @@ mod tests { .unwrap(); let table = write_batch(table, batch).await; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 1); let (table, metrics) = DeltaOps(table) @@ -635,7 +681,7 @@ mod tests { .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert_eq!(table.get_files_count(), 1); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); @@ -680,7 +726,7 @@ mod tests { // The expectation is that a physical scan of data is not required let table = write_batch(table, batch).await; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 1); let (table, metrics) = DeltaOps(table) @@ -690,7 +736,7 @@ mod tests { .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert_eq!(table.get_files_count(), 1); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); @@ -737,7 +783,7 @@ mod tests { .unwrap(); let table = write_batch(table, batch.clone()).await; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 2); let (table, metrics) = DeltaOps(table) @@ -748,7 +794,7 @@ mod tests { .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert_eq!(table.get_files_count(), 2); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); @@ -772,7 +818,7 @@ mod tests { // Update a partitioned table where the predicate contains a partition column and non-partition column let table = setup_table(Some(vec!["modified"])).await; let table = write_batch(table, batch).await; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 2); let (table, metrics) = DeltaOps(table) @@ -787,7 +833,7 @@ mod tests { .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert_eq!(table.get_files_count(), 3); assert_eq!(metrics.num_added_files, 2); assert_eq!(metrics.num_removed_files, 1); @@ -818,7 +864,7 @@ mod tests { true, ), StructField::new( - "ValUe".to_string(), + "ValUe".to_string(), // spellchecker:disable-line DeltaDataType::Primitive(PrimitiveType::Integer), true, ), @@ -831,7 +877,7 @@ mod tests { let arrow_schema = Arc::new(ArrowSchema::new(vec![ Field::new("Id", DataType::Utf8, true), - Field::new("ValUe", DataType::Int32, true), + Field::new("ValUe", DataType::Int32, true), // spellchecker:disable-line Field::new("mOdified", DataType::Utf8, true), ])); @@ -867,7 +913,7 @@ mod tests { let expected = vec![ "+----+-------+------------+", - "| Id | ValUe | mOdified |", + "| Id | ValUe | mOdified |", // spellchecker:disable-line "+----+-------+------------+", "| A | 1 | 2021-02-02 |", "| B | 10 | 2021-02-02 |", @@ -883,7 +929,7 @@ mod tests { #[tokio::test] async fn test_update_null() { let table = prepare_values_table().await; - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_files_count(), 1); let (table, metrics) = DeltaOps(table) @@ -891,7 +937,7 @@ mod tests { .with_update("value", col("value") + lit(1)) .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 1); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); @@ -921,7 +967,7 @@ mod tests { .with_update("value", lit(10)) .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 1); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); @@ -957,7 +1003,7 @@ mod tests { .with_update("value", "10") .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 1); assert_eq!(metrics.num_added_files, 1); assert_eq!(metrics.num_removed_files, 1); @@ -985,7 +1031,7 @@ mod tests { let table = prepare_values_table().await; let (table, metrics) = DeltaOps(table).update().await.unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(metrics.num_added_files, 0); assert_eq!(metrics.num_removed_files, 0); assert_eq!(metrics.num_copied_rows, 0); @@ -1001,7 +1047,7 @@ mod tests { .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(metrics.num_added_files, 0); assert_eq!(metrics.num_removed_files, 0); assert_eq!(metrics.num_copied_rows, 0); @@ -1055,7 +1101,7 @@ mod tests { true, ), ]); - let arrow_schema: ArrowSchema = (&schema).try_into().unwrap(); + let arrow_schema: ArrowSchema = (&schema).try_into_arrow().unwrap(); // Create the first batch let arrow_field = Field::new("element", DataType::Int32, false); @@ -1075,13 +1121,13 @@ mod tests { .with_columns(schema.fields().cloned()) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let table = DeltaOps(table) .write(vec![batch]) .await .expect("Failed to write first batch"); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); // Completed the first creation/write use arrow::array::{Int32Builder, ListBuilder}; @@ -1096,7 +1142,7 @@ mod tests { .with_update("items", lit(new_items)) .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); } /// Lists coming in from the Python bindings need to be parsed as SQL expressions by the update @@ -1125,7 +1171,7 @@ mod tests { true, ), ]); - let arrow_schema: ArrowSchema = (&schema).try_into().unwrap(); + let arrow_schema: ArrowSchema = (&schema).try_into_arrow().unwrap(); // Create the first batch let arrow_field = Field::new("element", DataType::Int64, true); @@ -1146,13 +1192,13 @@ mod tests { .with_columns(schema.fields().cloned()) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let table = DeltaOps(table) .write(vec![batch]) .await .expect("Failed to write first batch"); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); // Completed the first creation/write let (table, _metrics) = DeltaOps(table) @@ -1161,13 +1207,13 @@ mod tests { .with_update("items", "[100]".to_string()) .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); } #[tokio::test] async fn test_no_cdc_on_older_tables() { let table = prepare_values_table().await; - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_files_count(), 1); let schema = Arc::new(Schema::new(vec![Field::new( @@ -1184,7 +1230,7 @@ mod tests { .write(vec![batch]) .await .expect("Failed to write first batch"); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let (table, _metrics) = DeltaOps(table) .update() @@ -1192,7 +1238,7 @@ mod tests { .with_update("value", lit(12)) .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); // NOTE: This currently doesn't really assert anything because cdc_files() is not reading // actions correct @@ -1209,7 +1255,7 @@ mod tests { // Too close for missiles, switching to guns. Just checking that the data wasn't actually // written instead! - if let Ok(files) = crate::storage::utils::flatten_list_stream( + if let Ok(files) = crate::logstore::tests::flatten_list_stream( &table.object_store(), Some(&object_store::path::Path::from("_change_data")), ) @@ -1241,7 +1287,7 @@ mod tests { .with_configuration_property(TableProperty::EnableChangeDataFeed, Some("true")) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let schema = Arc::new(Schema::new(vec![Field::new( "value", @@ -1258,7 +1304,7 @@ mod tests { .write(vec![batch]) .await .expect("Failed to write first batch"); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let (table, _metrics) = DeltaOps(table) .update() @@ -1266,7 +1312,7 @@ mod tests { .with_update("value", lit(12)) .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); let ctx = SessionContext::new(); let table = DeltaOps(table) @@ -1325,7 +1371,7 @@ mod tests { .with_configuration_property(TableProperty::EnableChangeDataFeed, Some("true")) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let schema = Arc::new(Schema::new(vec![ Field::new("year", DataType::Utf8, true), @@ -1348,7 +1394,7 @@ mod tests { .write(vec![batch]) .await .expect("Failed to write first batch"); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let (table, _metrics) = DeltaOps(table) .update() @@ -1356,7 +1402,7 @@ mod tests { .with_update("year", "2024") .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); let ctx = SessionContext::new(); let table = DeltaOps(table) diff --git a/crates/core/src/operations/update_field_metadata.rs b/crates/core/src/operations/update_field_metadata.rs index b81382dc24..2062c97960 100644 --- a/crates/core/src/operations/update_field_metadata.rs +++ b/crates/core/src/operations/update_field_metadata.rs @@ -7,8 +7,8 @@ use delta_kernel::schema::{MetadataValue, StructType}; use futures::future::BoxFuture; use itertools::Itertools; -use super::transaction::{CommitBuilder, CommitProperties}; use super::{CustomExecuteHandler, Operation}; +use crate::kernel::transaction::{CommitBuilder, CommitProperties}; use crate::logstore::LogStoreRef; use crate::protocol::DeltaOperation; use crate::table::state::DeltaTableState; diff --git a/crates/core/src/operations/update_table_metadata.rs b/crates/core/src/operations/update_table_metadata.rs new file mode 100644 index 0000000000..cede54cc12 --- /dev/null +++ b/crates/core/src/operations/update_table_metadata.rs @@ -0,0 +1,153 @@ +//! Update table metadata operation + +use std::sync::Arc; + +use futures::future::BoxFuture; +use validator::Validate; + +use super::{CustomExecuteHandler, Operation}; +use crate::kernel::transaction::{CommitBuilder, CommitProperties}; +use crate::kernel::Action; +use crate::logstore::LogStoreRef; +use crate::protocol::DeltaOperation; +use crate::table::state::DeltaTableState; +use crate::DeltaTable; +use crate::{DeltaResult, DeltaTableError}; + +#[derive(Debug, Clone, serde::Serialize, serde::Deserialize, Validate)] +#[validate(schema( + function = "validate_at_least_one_field", + message = "No metadata update specified" +))] +pub struct TableMetadataUpdate { + #[validate(length( + min = 1, + max = 255, + message = "Table name cannot be empty and cannot exceed 255 characters" + ))] + pub name: Option, + + #[validate(length( + max = 4000, + message = "Table description cannot exceed 4000 characters" + ))] + pub description: Option, +} + +fn validate_at_least_one_field( + update: &TableMetadataUpdate, +) -> Result<(), validator::ValidationError> { + if update.name.is_none() && update.description.is_none() { + return Err(validator::ValidationError::new("no_fields_specified")); + } + Ok(()) +} + +/// Update table metadata operation +pub struct UpdateTableMetadataBuilder { + /// A snapshot of the table's state + snapshot: DeltaTableState, + /// The metadata update to apply + update: Option, + /// Delta object store for handling data files + log_store: LogStoreRef, + /// Additional information to add to the commit + commit_properties: CommitProperties, + custom_execute_handler: Option>, +} + +impl super::Operation<()> for UpdateTableMetadataBuilder { + fn log_store(&self) -> &LogStoreRef { + &self.log_store + } + fn get_custom_execute_handler(&self) -> Option> { + self.custom_execute_handler.clone() + } +} + +impl UpdateTableMetadataBuilder { + /// Create a new builder + pub fn new(log_store: LogStoreRef, snapshot: DeltaTableState) -> Self { + Self { + update: None, + snapshot, + log_store, + commit_properties: CommitProperties::default(), + custom_execute_handler: None, + } + } + + /// Specify the complete metadata update + pub fn with_update(mut self, update: TableMetadataUpdate) -> Self { + self.update = Some(update); + self + } + + /// Additional metadata to be added to commit info + pub fn with_commit_properties(mut self, commit_properties: CommitProperties) -> Self { + self.commit_properties = commit_properties; + self + } + + /// Set a custom execute handler, for pre and post execution + pub fn with_custom_execute_handler(mut self, handler: Arc) -> Self { + self.custom_execute_handler = Some(handler); + self + } +} + +impl std::future::IntoFuture for UpdateTableMetadataBuilder { + type Output = DeltaResult; + + type IntoFuture = BoxFuture<'static, Self::Output>; + + fn into_future(self) -> Self::IntoFuture { + let this = self; + + Box::pin(async move { + let operation_id = this.get_operation_id(); + this.pre_execute(operation_id).await?; + + let update = this.update.ok_or_else(|| { + DeltaTableError::MetadataError("No metadata update specified".to_string()) + })?; + update + .validate() + .map_err(|e| DeltaTableError::MetadataError(format!("{}", e)))?; + + let mut metadata = this.snapshot.metadata().clone(); + + if let Some(name) = &update.name { + metadata.name = Some(name.clone()); + } + if let Some(description) = &update.description { + metadata.description = Some(description.clone()); + } + + let operation = DeltaOperation::UpdateTableMetadata { + metadata_update: update, + }; + + let actions = vec![Action::Metadata(metadata)]; + + let commit = CommitBuilder::from(this.commit_properties.clone()) + .with_actions(actions) + .with_operation_id(operation_id) + .with_post_commit_hook_handler(this.custom_execute_handler.clone()) + .build( + Some(&this.snapshot), + this.log_store.clone(), + operation.clone(), + ) + .await?; + + if let Some(handler) = this.custom_execute_handler { + handler.post_execute(&this.log_store, operation_id).await?; + } + Ok(DeltaTable::new_with_state( + this.log_store, + commit.snapshot(), + )) + }) + } +} diff --git a/crates/core/src/operations/vacuum.rs b/crates/core/src/operations/vacuum.rs index 4e5c46589f..3455fed8e2 100644 --- a/crates/core/src/operations/vacuum.rs +++ b/crates/core/src/operations/vacuum.rs @@ -31,11 +31,12 @@ use futures::{StreamExt, TryStreamExt}; use object_store::Error; use object_store::{path::Path, ObjectStore}; use serde::Serialize; +use tracing::log::*; -use super::transaction::{CommitBuilder, CommitProperties}; use super::{CustomExecuteHandler, Operation}; use crate::errors::{DeltaResult, DeltaTableError}; -use crate::logstore::LogStoreRef; +use crate::kernel::transaction::{CommitBuilder, CommitProperties}; +use crate::logstore::{LogStore, LogStoreRef}; use crate::protocol::DeltaOperation; use crate::table::state::DeltaTableState; use crate::DeltaTable; @@ -57,9 +58,6 @@ enum VacuumError { /// Error returned #[error(transparent)] DeltaTable(#[from] DeltaTableError), - - #[error(transparent)] - Protocol(#[from] crate::protocol::ProtocolError), } impl From for DeltaTableError { @@ -76,6 +74,20 @@ pub trait Clock: Debug + Send + Sync { fn current_timestamp_millis(&self) -> i64; } +/// Type of Vacuum operation to perform +#[derive(Debug, Default, Clone, PartialEq)] +pub enum VacuumMode { + /// The `lite` mode will only remove files which are referenced in the `_delta_log` associated + /// with `remove` action + #[default] + Lite, + /// A `full` mode vacuum will remove _all_ data files no longer actively referenced in the + /// `_delta_log` table. For example, if parquet files exist in the table directory but are no + /// longer mentioned as `add` actions in the transaction log, then this mode will scan storage + /// and remove those files. + Full, +} + /// Vacuum a Delta table with the given options /// See this module's documentation for more information pub struct VacuumBuilder { @@ -89,6 +101,8 @@ pub struct VacuumBuilder { enforce_retention_duration: bool, /// Don't delete the files. Just determine which files can be deleted dry_run: bool, + /// Mode of vacuum that should be run + mode: VacuumMode, /// Override the source of time clock: Option>, /// Additional information to add to the commit @@ -144,6 +158,7 @@ impl VacuumBuilder { retention_period: None, enforce_retention_duration: true, dry_run: false, + mode: VacuumMode::Lite, clock: None, commit_properties: CommitProperties::default(), custom_execute_handler: None, @@ -156,6 +171,12 @@ impl VacuumBuilder { self } + /// Override the default vacuum mode (lite) + pub fn with_mode(mut self, mode: VacuumMode) -> Self { + self.mode = mode; + self + } + /// Only determine which files should be deleted pub fn with_dry_run(mut self, dry_run: bool) -> Self { self.dry_run = dry_run; @@ -189,6 +210,10 @@ impl VacuumBuilder { /// Determine which files can be deleted. Does not actually perform the deletion async fn create_vacuum_plan(&self) -> Result { + if self.mode == VacuumMode::Full { + info!("Vacuum configured to run with 'VacuumMode::Full'. It will scan for orphaned parquet files in the Delta table directory and remove those as well!"); + } + let min_retention = Duration::milliseconds( self.snapshot .table_config() @@ -214,7 +239,7 @@ impl VacuumBuilder { &self.snapshot, retention_period, now_millis, - self.log_store.object_store(None).clone(), + &self.log_store, ) .await?; let valid_files = self.snapshot.file_paths_iter().collect::>(); @@ -228,12 +253,24 @@ impl VacuumBuilder { while let Some(obj_meta) = all_files.next().await { // TODO should we allow NotFound here in case we have a temporary commit file in the list let obj_meta = obj_meta.map_err(DeltaTableError::from)?; - if valid_files.contains(&obj_meta.location) // file is still being tracked in table - || !expired_tombstones.contains(obj_meta.location.as_ref()) // file is not an expired tombstone - || is_hidden_directory(partition_columns, &obj_meta.location)? - { + // file is still being tracked in table + if valid_files.contains(&obj_meta.location) { + continue; + } + if is_hidden_directory(partition_columns, &obj_meta.location)? { continue; } + // file is not an expired tombstone _and_ this is a "Lite" vacuum + // If the file is not an expired tombstone and we have gotten to here with a + // VacuumMode::Full then it should be added to the deletion plan + if !expired_tombstones.contains(obj_meta.location.as_ref()) { + if self.mode == VacuumMode::Lite { + debug!("The file {:?} was not referenced in a log file, but VacuumMode::Lite means it will not be vacuumed", &obj_meta.location); + continue; + } else { + debug!("The file {:?} was not referenced in a log file, but VacuumMode::Full means it *will be vacuumed*", &obj_meta.location); + } + } files_to_delete.push(obj_meta.location); file_sizes.push(obj_meta.size as i64); @@ -412,7 +449,7 @@ async fn get_stale_files( snapshot: &DeltaTableState, retention_period: Duration, now_timestamp_millis: i64, - store: Arc, + store: &dyn LogStore, ) -> DeltaResult> { let tombstone_retention_timestamp = now_timestamp_millis - retention_period.num_milliseconds(); Ok(snapshot @@ -436,7 +473,44 @@ mod tests { use std::time::SystemTime; #[tokio::test] - async fn vacuum_delta_8_0_table() { + async fn test_vacuum_full() -> DeltaResult<()> { + let table = open_table("../test/tests/data/simple_commit").await?; + + let (_table, result) = VacuumBuilder::new(table.log_store(), table.snapshot()?.clone()) + .with_retention_period(Duration::hours(0)) + .with_dry_run(true) + .with_mode(VacuumMode::Lite) + .with_enforce_retention_duration(false) + .await?; + // When running lite, this table with superfluous parquet files should not have anything to + // delete + assert!(result.files_deleted.is_empty()); + + let (_table, result) = VacuumBuilder::new(table.log_store(), table.snapshot()?.clone()) + .with_retention_period(Duration::hours(0)) + .with_dry_run(true) + .with_mode(VacuumMode::Full) + .with_enforce_retention_duration(false) + .await?; + let mut files_deleted = result.files_deleted.clone(); + files_deleted.sort(); + // When running with full, these superfluous parquet files which are not actually + // referenced in the _delta_log commits should be considered for the + // low-orbit ion-cannon + assert_eq!( + files_deleted, + vec![ + "part-00000-512e1537-8aaa-4193-b8b4-bef3de0de409-c000.snappy.parquet", + "part-00000-b44fcdb0-8b06-4f3a-8606-f8311a96f6dc-c000.snappy.parquet", + "part-00001-185eca06-e017-4dea-ae49-fc48b973e37e-c000.snappy.parquet", + "part-00001-4327c977-2734-4477-9507-7ccf67924649-c000.snappy.parquet", + ] + ); + Ok(()) + } + + #[tokio::test] + async fn vacuum_delta_8_0_table() -> DeltaResult<()> { let table = open_table("../test/tests/data/delta-0.8.0").await.unwrap(); let result = VacuumBuilder::new(table.log_store(), table.snapshot().unwrap().clone()) @@ -453,8 +527,7 @@ mod tests { .with_retention_period(Duration::hours(0)) .with_dry_run(true) .with_enforce_retention_duration(false) - .await - .unwrap(); + .await?; // do not enforce retention duration check with 0 hour will purge all files assert_eq!( result.files_deleted, @@ -465,8 +538,7 @@ mod tests { VacuumBuilder::new(table.log_store(), table.snapshot().unwrap().clone()) .with_retention_period(Duration::hours(169)) .with_dry_run(true) - .await - .unwrap(); + .await?; assert_eq!( result.files_deleted, @@ -483,9 +555,9 @@ mod tests { VacuumBuilder::new(table.log_store(), table.snapshot().unwrap().clone()) .with_retention_period(Duration::hours(retention_hours as i64)) .with_dry_run(true) - .await - .unwrap(); + .await?; assert_eq!(result.files_deleted, empty); + Ok(()) } } diff --git a/crates/core/src/operations/write/async_utils.rs b/crates/core/src/operations/write/async_utils.rs index 9fac4b062a..0d35deee9d 100644 --- a/crates/core/src/operations/write/async_utils.rs +++ b/crates/core/src/operations/write/async_utils.rs @@ -28,6 +28,7 @@ impl AsyncShareableBuffer { } /// Returns a clone of the underlying buffer as a `Vec`. + #[allow(dead_code)] pub async fn to_vec(&self) -> Vec { let inner = self.buffer.read().await; inner.clone() @@ -40,12 +41,14 @@ impl AsyncShareableBuffer { } /// Returns `true` if the underlying buffer is empty. + #[allow(dead_code)] pub async fn is_empty(&self) -> bool { let inner = self.buffer.read().await; inner.is_empty() } /// Creates a new instance with the buffer initialized from the provided bytes. + #[allow(dead_code)] pub fn from_bytes(bytes: &[u8]) -> Self { Self { buffer: Arc::new(TokioRwLock::new(bytes.to_vec())), diff --git a/crates/core/src/operations/write/execution.rs b/crates/core/src/operations/write/execution.rs index 73e24aef03..5e8e51f77c 100644 --- a/crates/core/src/operations/write/execution.rs +++ b/crates/core/src/operations/write/execution.rs @@ -7,6 +7,7 @@ use datafusion::execution::context::{SessionState, TaskContext}; use datafusion::prelude::DataFrame; use datafusion_expr::{lit, when, Expr, LogicalPlanBuilder}; use datafusion_physical_plan::ExecutionPlan; +use delta_kernel::engine::arrow_conversion::TryIntoKernel as _; use futures::StreamExt; use object_store::prefix::PrefixStore; use parquet::file::properties::WriterProperties; @@ -19,9 +20,8 @@ use crate::delta_datafusion::{find_files, DeltaScanConfigBuilder, DeltaTableProv use crate::delta_datafusion::{DataFusionMixins, DeltaDataChecker}; use crate::errors::DeltaResult; use crate::kernel::{Action, Add, AddCDCFile, Remove, StructType, StructTypeExt}; -use crate::logstore::LogStoreRef; +use crate::logstore::{LogStoreRef, ObjectStoreRef}; use crate::operations::cdc::should_write_cdc; -use crate::storage::ObjectStoreRef; use crate::table::state::DeltaTableState; use crate::table::Constraint as DeltaConstraint; use crate::DeltaTableError; @@ -258,7 +258,7 @@ pub(crate) async fn write_execution_plan_v2( DeltaDataChecker::new(snapshot) } else { debug!("Using plan schema to derive generated columns, since no snapshot was provided. Implies first write."); - let delta_schema: StructType = schema.as_ref().try_into()?; + let delta_schema: StructType = schema.as_ref().try_into_kernel()?; DeltaDataChecker::new_with_generated_columns( delta_schema.get_generated_columns().unwrap_or_default(), ) diff --git a/crates/core/src/operations/write/generated_columns.rs b/crates/core/src/operations/write/generated_columns.rs index 582571afaa..6081455ea2 100644 --- a/crates/core/src/operations/write/generated_columns.rs +++ b/crates/core/src/operations/write/generated_columns.rs @@ -1,10 +1,27 @@ +use crate::table::state::DeltaTableState; use datafusion::{execution::SessionState, prelude::DataFrame}; use datafusion_common::ScalarValue; use datafusion_expr::{col, when, Expr, ExprSchemable}; +use delta_kernel::engine::arrow_conversion::TryIntoArrow as _; use tracing::debug; use crate::{kernel::DataCheck, table::GeneratedColumn, DeltaResult}; +/// check if the writer version is able to write generated columns +pub fn able_to_gc(snapshot: &DeltaTableState) -> DeltaResult { + if let Some(features) = &snapshot.protocol().writer_features { + if snapshot.protocol().min_writer_version < 4 { + return Ok(false); + } + if snapshot.protocol().min_writer_version == 7 + && !features.contains(&delta_kernel::table_features::WriterFeature::GeneratedColumns) + { + return Ok(false); + } + } + Ok(true) +} + /// Add generated column expressions to a dataframe pub fn add_missing_generated_columns( mut df: DataFrame, @@ -59,10 +76,7 @@ pub fn add_generated_columns( generated_col.get_name(), when(col(col_name).is_null(), generation_expr) .otherwise(col(col_name))? - .cast_to( - &arrow_schema::DataType::try_from(&generated_col.data_type)?, - df.schema(), - )?, + .cast_to(&((&generated_col.data_type).try_into_arrow()?), df.schema())?, )? } Ok(df) diff --git a/crates/core/src/operations/write/mod.rs b/crates/core/src/operations/write/mod.rs index 8d3560e561..fc40ba9652 100644 --- a/crates/core/src/operations/write/mod.rs +++ b/crates/core/src/operations/write/mod.rs @@ -31,10 +31,11 @@ pub(crate) mod metrics; pub(crate) mod schema_evolution; pub mod writer; -use arrow_schema::{DataType, Schema}; +use arrow_schema::Schema; pub use configs::WriterStatsConfig; use datafusion::execution::SessionStateBuilder; -use generated_columns::{add_generated_columns, add_missing_generated_columns}; +use delta_kernel::engine::arrow_conversion::TryIntoKernel as _; +use generated_columns::{able_to_gc, add_generated_columns, add_missing_generated_columns}; use metrics::{WriteMetricExtensionPlanner, SOURCE_COUNT_ID, SOURCE_COUNT_METRIC}; use std::collections::HashMap; use std::str::FromStr; @@ -58,7 +59,6 @@ use tracing::log::*; use super::cdc::CDC_COLUMN_NAME; use super::datafusion_utils::Expression; -use super::transaction::{CommitBuilder, CommitProperties, TableReference, PROTOCOL}; use super::{CreateBuilder, CustomExecuteHandler, Operation}; use crate::delta_datafusion::expr::fmt_expr_to_sql; use crate::delta_datafusion::expr::parse_predicate_expression; @@ -68,6 +68,7 @@ use crate::delta_datafusion::planner::DeltaPlanner; use crate::delta_datafusion::register_store; use crate::delta_datafusion::DataFusionMixins; use crate::errors::{DeltaResult, DeltaTableError}; +use crate::kernel::transaction::{CommitBuilder, CommitProperties, TableReference, PROTOCOL}; use crate::kernel::{Action, ActionType, Metadata, StructType, StructTypeExt}; use crate::logstore::LogStoreRef; use crate::operations::cast::merge_schema::merge_arrow_schema; @@ -360,7 +361,7 @@ impl WriteBuilder { .input .clone() .ok_or::(WriteError::MissingData.into())?; - let schema: StructType = input.schema().as_arrow().try_into()?; + let schema: StructType = input.schema().as_arrow().try_into_kernel()?; match &self.snapshot { Some(snapshot) => { @@ -445,18 +446,21 @@ impl std::future::IntoFuture for WriteBuilder { state } }; - let generated_col_expressions = this - .snapshot - .as_ref() - .map(|v| v.schema().get_generated_columns().unwrap_or_default()) - .unwrap_or_default(); - let mut schema_drift = false; - let source = DataFrame::new(state.clone(), this.input.unwrap().as_ref().clone()); - - // Add missing generated columns to source_df - let (mut source, missing_generated_columns) = - add_missing_generated_columns(source, &generated_col_expressions)?; + let mut generated_col_exp = None; + let mut missing_gen_col = None; + let mut source = DataFrame::new(state.clone(), this.input.unwrap().as_ref().clone()); + if let Some(snapshot) = &this.snapshot { + if able_to_gc(snapshot)? { + let generated_col_expressions = snapshot.schema().get_generated_columns()?; + // Add missing generated columns to source_df + let (source_with_gc, missing_generated_columns) = + add_missing_generated_columns(source, &generated_col_expressions)?; + source = source_with_gc; + missing_gen_col = Some(missing_generated_columns); + generated_col_exp = Some(generated_col_expressions); + } + } let source_schema: Arc = Arc::new(source.schema().as_arrow().clone()); @@ -527,12 +531,16 @@ impl std::future::IntoFuture for WriteBuilder { source = source.select(schema_evolution_projection)?; } - source = add_generated_columns( - source, - &generated_col_expressions, - &missing_generated_columns, - &state, - )?; + if let Some(generated_columns_exp) = generated_col_exp { + if let Some(missing_generated_col) = missing_gen_col { + source = add_generated_columns( + source, + &generated_columns_exp, + &missing_generated_col, + &state, + )?; + } + } let source = LogicalPlan::Extension(Extension { node: Arc::new(MetricObserver { @@ -549,7 +557,7 @@ impl std::future::IntoFuture for WriteBuilder { // Maybe create schema action if this.schema_mode == Some(SchemaMode::Merge) && schema_drift { if let Some(snapshot) = &this.snapshot { - let schema_struct: StructType = schema.clone().try_into()?; + let schema_struct: StructType = schema.clone().try_into_kernel()?; // Verify if delta schema changed if &schema_struct != snapshot.schema() { let current_protocol = snapshot.protocol(); @@ -585,7 +593,6 @@ impl std::future::IntoFuture for WriteBuilder { Expression::String(s) => { let df_schema = DFSchema::try_from(schema.as_ref().to_owned())?; parse_predicate_expression(&df_schema, s, &state)? - // this.snapshot.unwrap().parse_predicate_expression(s, &state)? } }; (Some(fmt_expr_to_sql(&pred)?), Some(pred)) @@ -614,7 +621,7 @@ impl std::future::IntoFuture for WriteBuilder { // Collect remove actions if we are overwriting the table if let Some(snapshot) = &this.snapshot { if matches!(this.mode, SaveMode::Overwrite) { - let delta_schema: StructType = schema.as_ref().try_into()?; + let delta_schema: StructType = schema.as_ref().try_into_kernel()?; // Update metadata with new schema if there is a change if &delta_schema != snapshot.schema() { let mut metadata = snapshot.metadata().clone(); @@ -766,6 +773,7 @@ mod tests { use arrow_schema::{DataType, Field, Fields, Schema as ArrowSchema, TimeUnit}; use datafusion::prelude::*; use datafusion::{assert_batches_eq, assert_batches_sorted_eq}; + use delta_kernel::engine::arrow_conversion::TryIntoArrow; use itertools::Itertools; use serde_json::{json, Value}; @@ -814,7 +822,7 @@ mod tests { .with_columns(table_schema.fields().cloned()) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.history(None).await.unwrap().len(), 1); // write some data @@ -825,7 +833,7 @@ mod tests { .with_commit_properties(CommitProperties::default().with_metadata(metadata.clone())) .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert_eq!(table.get_files_count(), 1); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; @@ -854,7 +862,7 @@ mod tests { .with_commit_properties(CommitProperties::default().with_metadata(metadata.clone())) .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); assert_eq!(table.get_files_count(), 2); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; assert_eq!(write_metrics.num_added_rows, batch.num_rows()); @@ -882,7 +890,7 @@ mod tests { .with_commit_properties(CommitProperties::default().with_metadata(metadata.clone())) .await .unwrap(); - assert_eq!(table.version(), 3); + assert_eq!(table.version(), Some(3)); assert_eq!(table.get_files_count(), 1); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; assert_eq!(write_metrics.num_added_rows, batch.num_rows()); @@ -1021,7 +1029,7 @@ mod tests { .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_files_count(), 1); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; assert_common_write_metrics(write_metrics); @@ -1036,7 +1044,7 @@ mod tests { .with_partition_columns(["modified"]) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_files_count(), 2); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; assert_eq!(write_metrics.num_added_files, 2); @@ -1048,7 +1056,7 @@ mod tests { .with_partition_columns(["modified", "id"]) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_files_count(), 4); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; @@ -1064,7 +1072,7 @@ mod tests { .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; assert_common_write_metrics(write_metrics); @@ -1110,7 +1118,7 @@ mod tests { .await .unwrap(); table.load().await.unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let new_schema = table.metadata().unwrap().schema().unwrap(); let fields = new_schema.fields(); let names = fields.map(|f| f.name()).collect::>(); @@ -1138,7 +1146,7 @@ mod tests { .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; assert_common_write_metrics(write_metrics); @@ -1183,7 +1191,7 @@ mod tests { .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let new_schema = table.metadata().unwrap().schema().unwrap(); let fields = new_schema.fields(); let mut names = fields.map(|f| f.name()).collect::>(); @@ -1204,7 +1212,7 @@ mod tests { .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; assert_common_write_metrics(write_metrics); let mut new_schema_builder = arrow_schema::SchemaBuilder::new(); @@ -1258,7 +1266,7 @@ mod tests { .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; assert_common_write_metrics(write_metrics); @@ -1312,10 +1320,10 @@ mod tests { .with_columns(schema.fields().cloned()) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let table = DeltaOps(table).write(vec![batch.clone()]).await.unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; assert_common_write_metrics(write_metrics); @@ -1336,7 +1344,7 @@ mod tests { .with_columns(schema.fields().cloned()) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let table = DeltaOps(table).write(vec![batch.clone()]).await; assert!(table.is_err()); @@ -1352,14 +1360,14 @@ mod tests { .with_columns(table_schema.fields().cloned()) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let table = DeltaOps(table) .write(vec![batch.clone()]) .with_save_mode(SaveMode::Append) .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; assert_common_write_metrics(write_metrics); @@ -1444,7 +1452,7 @@ mod tests { .with_save_mode(SaveMode::Append) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; assert_eq!(write_metrics.num_added_rows, 4); assert_common_write_metrics(write_metrics); @@ -1465,7 +1473,7 @@ mod tests { .with_replace_where(col("id").eq(lit("C"))) .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; assert_eq!(write_metrics.num_added_rows, 1); assert_common_write_metrics(write_metrics); @@ -1506,7 +1514,7 @@ mod tests { .with_save_mode(SaveMode::Append) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; assert_common_write_metrics(write_metrics); @@ -1550,7 +1558,7 @@ mod tests { .with_save_mode(SaveMode::Append) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; assert_common_write_metrics(write_metrics); @@ -1574,7 +1582,7 @@ mod tests { .with_replace_where(col("id").eq(lit("A"))) .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; assert_eq!(write_metrics.num_added_rows, 3); assert_common_write_metrics(write_metrics); @@ -1606,9 +1614,9 @@ mod tests { .with_configuration_property(TableProperty::EnableChangeDataFeed, Some("true")) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); - let schema = Arc::new(ArrowSchema::try_from(delta_schema)?); + let schema: Arc = Arc::new(delta_schema.try_into_arrow()?); let batch = RecordBatch::try_new( Arc::clone(&schema), @@ -1638,7 +1646,7 @@ mod tests { .write(vec![batch]) .await .expect("Failed to write first batch"); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; assert_eq!(write_metrics.num_added_rows, 3); assert_common_write_metrics(write_metrics); @@ -1648,7 +1656,7 @@ mod tests { .with_save_mode(crate::protocol::SaveMode::Overwrite) .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; assert_eq!(write_metrics.num_added_rows, 1); assert!(write_metrics.num_removed_files > 0); @@ -1679,9 +1687,9 @@ mod tests { .with_configuration_property(TableProperty::EnableChangeDataFeed, Some("true")) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); - let schema = Arc::new(ArrowSchema::try_from(delta_schema)?); + let schema: Arc = Arc::new(delta_schema.try_into_arrow()?); let batch = RecordBatch::try_new( Arc::clone(&schema), @@ -1711,7 +1719,7 @@ mod tests { .write(vec![batch]) .await .expect("Failed to write first batch"); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; assert_eq!(write_metrics.num_added_rows, 3); assert_common_write_metrics(write_metrics); @@ -1722,7 +1730,7 @@ mod tests { .with_replace_where("id='3'") .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); let write_metrics: WriteMetrics = get_write_metrics(table.clone()).await; assert_eq!(write_metrics.num_added_rows, 1); assert!(write_metrics.num_removed_files > 0); @@ -1753,9 +1761,9 @@ mod tests { .with_configuration_property(TableProperty::EnableChangeDataFeed, Some("true")) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); - let schema = Arc::new(ArrowSchema::try_from(delta_schema)?); + let schema: Arc = Arc::new(delta_schema.try_into_arrow()?); let batch = RecordBatch::try_new( Arc::clone(&schema), @@ -1785,7 +1793,7 @@ mod tests { .write(vec![batch]) .await .expect("Failed to write first batch"); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let table = DeltaOps(table) .write([second_batch]) @@ -1793,7 +1801,7 @@ mod tests { .with_replace_where("value=3") .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); let ctx = SessionContext::new(); let cdf_scan = DeltaOps(table.clone()) diff --git a/crates/core/src/operations/write/writer.rs b/crates/core/src/operations/write/writer.rs index a8e022b4a6..edc5784e6e 100644 --- a/crates/core/src/operations/write/writer.rs +++ b/crates/core/src/operations/write/writer.rs @@ -20,7 +20,7 @@ use super::async_utils::AsyncShareableBuffer; use crate::crate_version; use crate::errors::{DeltaResult, DeltaTableError}; use crate::kernel::{Add, PartitionsExt}; -use crate::storage::ObjectStoreRef; +use crate::logstore::ObjectStoreRef; use crate::writer::record_batch::{divide_by_partition_values, PartitionResult}; use crate::writer::stats::create_add; use crate::writer::utils::{ @@ -486,7 +486,7 @@ impl PartitionWriter { #[cfg(test)] mod tests { use super::*; - use crate::storage::utils::flatten_list_stream as list; + use crate::logstore::tests::flatten_list_stream as list; use crate::table::config::DEFAULT_NUM_INDEX_COLS; use crate::writer::test_utils::*; use crate::DeltaTableBuilder; @@ -534,7 +534,7 @@ mod tests { #[tokio::test] async fn test_write_partition() { - let log_store = DeltaTableBuilder::from_uri("memory://") + let log_store = DeltaTableBuilder::from_uri("memory:///") .build_storage() .unwrap(); let object_store = log_store.object_store(None); @@ -553,7 +553,7 @@ mod tests { .head(&Path::from(adds[0].path.clone())) .await .unwrap(); - assert_eq!(head.size, adds[0].size as usize) + assert_eq!(head.size, adds[0].size as u64) } #[tokio::test] @@ -566,7 +566,7 @@ mod tests { ])); let batch = RecordBatch::try_new(schema, vec![base_str, base_int]).unwrap(); - let object_store = DeltaTableBuilder::from_uri("memory://") + let object_store = DeltaTableBuilder::from_uri("memory:///") .build_storage() .unwrap() .object_store(None); @@ -597,7 +597,7 @@ mod tests { ])); let batch = RecordBatch::try_new(schema, vec![base_str, base_int]).unwrap(); - let object_store = DeltaTableBuilder::from_uri("memory://") + let object_store = DeltaTableBuilder::from_uri("memory:///") .build_storage() .unwrap() .object_store(None); @@ -624,7 +624,7 @@ mod tests { ])); let batch = RecordBatch::try_new(schema, vec![base_str, base_int]).unwrap(); - let object_store = DeltaTableBuilder::from_uri("memory://") + let object_store = DeltaTableBuilder::from_uri("memory:///") .build_storage() .unwrap() .object_store(None); @@ -639,7 +639,7 @@ mod tests { #[tokio::test] async fn test_write_mismatched_schema() { - let log_store = DeltaTableBuilder::from_uri("memory://") + let log_store = DeltaTableBuilder::from_uri("memory:///") .build_storage() .unwrap(); let object_store = log_store.object_store(None); diff --git a/crates/core/src/protocol/checkpoints.rs b/crates/core/src/protocol/checkpoints.rs index e7e3141cd7..69ccb13905 100644 --- a/crates/core/src/protocol/checkpoints.rs +++ b/crates/core/src/protocol/checkpoints.rs @@ -1,100 +1,111 @@ //! Implementation for writing delta checkpoints. -use std::collections::HashMap; -use std::iter::Iterator; use std::sync::LazyLock; -use arrow_json::ReaderBuilder; -use arrow_schema::ArrowError; - -use chrono::{Datelike, NaiveDate, NaiveDateTime, Utc}; +use arrow::compute::filter_record_batch; +use arrow_array::{BooleanArray, RecordBatch}; +use chrono::Utc; +use delta_kernel::engine::arrow_data::ArrowEngineData; +use delta_kernel::engine_data::FilteredEngineData; +use delta_kernel::snapshot::LastCheckpointHint; +use delta_kernel::{FileMeta, Table}; use futures::{StreamExt, TryStreamExt}; -use itertools::Itertools; +use object_store::path::Path; use object_store::{Error, ObjectStore}; -use parquet::arrow::ArrowWriter; -use parquet::basic::Compression; -use parquet::basic::Encoding; -use parquet::errors::ParquetError; -use parquet::file::properties::WriterProperties; +use parquet::arrow::async_writer::ParquetObjectWriter; +use parquet::arrow::AsyncArrowWriter; use regex::Regex; -use serde_json::Value; -use tracing::{debug, error}; +use tokio::task::spawn_blocking; +use tracing::{debug, error, warn}; use uuid::Uuid; -use super::{time_utils, ProtocolError}; -use crate::kernel::arrow::delta_log_schema_for_table; -use crate::kernel::{ - Action, Add as AddAction, DataType, PrimitiveType, Protocol, Remove, StructField, -}; -use crate::logstore::LogStore; -use crate::table::state::DeltaTableState; -use crate::table::{get_partition_col_data_types, CheckPoint, CheckPointBuilder}; +use crate::logstore::{LogStore, LogStoreExt}; use crate::{open_table_with_version, DeltaTable}; +use crate::{DeltaResult, DeltaTableError}; -type SchemaPath = Vec; - -/// Error returned when there is an error during creating a checkpoint. -#[derive(thiserror::Error, Debug)] -enum CheckpointError { - /// Error returned when a string formatted partition value cannot be parsed to its appropriate - /// data type. - #[error("Partition value {0} cannot be parsed from string.")] - PartitionValueNotParseable(String), - - /// Caller attempt to create a checkpoint for a version which does not exist on the table state - #[error("Attempted to create a checkpoint for a version {0} that does not match the table state {1}")] - StaleTableVersion(i64, i64), - - /// Error returned when the parquet writer fails while writing the checkpoint. - #[error("Failed to write parquet: {}", .source)] - Parquet { - /// Parquet error details returned when writing the checkpoint failed. - #[from] - source: ParquetError, - }, - - /// Error returned when converting the schema to Arrow format failed. - #[error("Failed to convert into Arrow schema: {}", .source)] - Arrow { - /// Arrow error details returned when converting the schema in Arrow format failed - #[from] - source: ArrowError, - }, - - #[error("missing required action type in snapshot: {0}")] - MissingActionType(String), -} +/// Creates checkpoint for a given table version, table state and object store +pub(crate) async fn create_checkpoint_for( + version: u64, + log_store: &dyn LogStore, + operation_id: Option, +) -> DeltaResult<()> { + let table_root = if let Some(op_id) = operation_id { + #[allow(deprecated)] + log_store.transaction_url(op_id, &log_store.table_root_url())? + } else { + log_store.table_root_url() + }; + let engine = log_store.engine(operation_id).await; -impl From for ProtocolError { - fn from(value: CheckpointError) -> Self { - match value { - CheckpointError::PartitionValueNotParseable(_) => Self::InvalidField(value.to_string()), - CheckpointError::Arrow { source } => Self::Arrow { source }, - CheckpointError::StaleTableVersion(..) => Self::Generic(value.to_string()), - CheckpointError::Parquet { source } => Self::ParquetParseError { source }, - CheckpointError::MissingActionType(_) => Self::Generic(value.to_string()), - } - } -} + let table = Table::try_from_uri(table_root)?; -use core::str::Utf8Error; -impl From for ProtocolError { - fn from(value: Utf8Error) -> Self { - Self::Generic(value.to_string()) + let task_engine = engine.clone(); + let cp_writer = spawn_blocking(move || table.checkpoint(task_engine.as_ref(), Some(version))) + .await + .map_err(|e| DeltaTableError::Generic(e.to_string()))??; + + let cp_url = cp_writer.checkpoint_path()?; + let cp_path = Path::from_url_path(cp_url.path())?; + let mut cp_data = cp_writer.checkpoint_data(engine.as_ref())?; + + let (first_batch, mut cp_data) = spawn_blocking(move || { + let Some(first_batch) = cp_data.next() else { + return Err(DeltaTableError::Generic("No data".to_string())); + }; + Ok((to_rb(first_batch?)?, cp_data)) + }) + .await + .map_err(|e| DeltaTableError::Generic(e.to_string()))??; + + let root_store = log_store.root_object_store(operation_id); + let object_store_writer = ParquetObjectWriter::new(root_store.clone(), cp_path.clone()); + let mut writer = AsyncArrowWriter::try_new(object_store_writer, first_batch.schema(), None)?; + writer.write(&first_batch).await?; + + let mut current_batch; + loop { + (current_batch, cp_data) = spawn_blocking(move || { + let Some(first_batch) = cp_data.next() else { + return Ok::<_, DeltaTableError>((None, cp_data)); + }; + Ok((Some(to_rb(first_batch?)?), cp_data)) + }) + .await + .map_err(|e| DeltaTableError::Generic(e.to_string()))??; + + let Some(batch) = current_batch else { + break; + }; + writer.write(&batch).await?; } + + let _pq_meta = writer.close().await?; + let file_meta = root_store.head(&cp_path).await?; + let file_meta = FileMeta { + location: cp_url, + size: file_meta.size, + last_modified: file_meta.last_modified.timestamp_millis(), + }; + + spawn_blocking(move || cp_writer.finalize(engine.as_ref(), &file_meta, cp_data)) + .await + .map_err(|e| DeltaTableError::Generic(e.to_string()))??; + + Ok(()) } -/// The record batch size for checkpoint parquet file -pub const CHECKPOINT_RECORD_BATCH_SIZE: usize = 5000; +fn to_rb(data: FilteredEngineData) -> DeltaResult { + let engine_data = ArrowEngineData::try_from_engine_data(data.data)?; + let predicate = BooleanArray::from(data.selection_vector); + let batch = filter_record_batch(engine_data.record_batch(), &predicate)?; + Ok(batch) +} /// Creates checkpoint at current table version -pub async fn create_checkpoint( - table: &DeltaTable, - operation_id: Option, -) -> Result<(), ProtocolError> { +pub async fn create_checkpoint(table: &DeltaTable, operation_id: Option) -> DeltaResult<()> { + let snapshot = table.snapshot()?; create_checkpoint_for( - table.version(), - table.snapshot().map_err(|_| ProtocolError::NoMetaData)?, + snapshot.version() as u64, table.log_store.as_ref(), operation_id, ) @@ -107,16 +118,12 @@ pub async fn create_checkpoint( pub async fn cleanup_metadata( table: &DeltaTable, operation_id: Option, -) -> Result { +) -> DeltaResult { + let snapshot = table.snapshot()?; let log_retention_timestamp = Utc::now().timestamp_millis() - - table - .snapshot() - .map_err(|_| ProtocolError::NoMetaData)? - .table_config() - .log_retention_duration() - .as_millis() as i64; + - snapshot.table_config().log_retention_duration().as_millis() as i64; cleanup_expired_logs_for( - table.version(), + snapshot.version(), table.log_store.as_ref(), log_retention_timestamp, operation_id, @@ -132,17 +139,15 @@ pub async fn create_checkpoint_from_table_uri_and_cleanup( version: i64, cleanup: Option, operation_id: Option, -) -> Result<(), ProtocolError> { - let table = open_table_with_version(table_uri, version) - .await - .map_err(|err| ProtocolError::Generic(err.to_string()))?; - let snapshot = table.snapshot().map_err(|_| ProtocolError::NoMetaData)?; - create_checkpoint_for(version, snapshot, table.log_store.as_ref(), None).await?; +) -> DeltaResult<()> { + let table = open_table_with_version(table_uri, version).await?; + let snapshot = table.snapshot()?; + create_checkpoint_for(version as u64, table.log_store.as_ref(), operation_id).await?; let enable_expired_log_cleanup = cleanup.unwrap_or_else(|| snapshot.table_config().enable_expired_log_cleanup()); - if table.version() >= 0 && enable_expired_log_cleanup { + if snapshot.version() >= 0 && enable_expired_log_cleanup { let deleted_log_num = cleanup_metadata(&table, operation_id).await?; debug!("Deleted {deleted_log_num:?} log files."); } @@ -150,60 +155,6 @@ pub async fn create_checkpoint_from_table_uri_and_cleanup( Ok(()) } -/// Creates checkpoint for a given table version, table state and object store -pub async fn create_checkpoint_for( - version: i64, - state: &DeltaTableState, - log_store: &dyn LogStore, - operation_id: Option, -) -> Result<(), ProtocolError> { - if !state.load_config().require_files { - return Err(ProtocolError::Generic( - "Table has not yet been initialized with files, therefore creating a checkpoint is not possible.".to_string() - )); - } - - if version != state.version() { - error!( - "create_checkpoint_for called with version {version} but table state contains: {}. The table state may need to be reloaded", - state.version() - ); - return Err(CheckpointError::StaleTableVersion(version, state.version()).into()); - } - - // TODO: checkpoints _can_ be multi-part... haven't actually found a good reference for - // an appropriate split point yet though so only writing a single part currently. - // See https://github.com/delta-io/delta-rs/issues/288 - let last_checkpoint_path = log_store.log_path().child("_last_checkpoint"); - - debug!("Writing parquet bytes to checkpoint buffer."); - let tombstones = state - .unexpired_tombstones(log_store.object_store(None).clone()) - .await - .map_err(|_| ProtocolError::Generic("filed to get tombstones".into()))? - .collect::>(); - let (checkpoint, parquet_bytes) = parquet_bytes_from_state(state, tombstones)?; - - let file_name = format!("{version:020}.checkpoint.parquet"); - let checkpoint_path = log_store.log_path().child(file_name); - - let object_store = log_store.object_store(operation_id); - debug!("Writing checkpoint to {checkpoint_path:?}."); - object_store - .put(&checkpoint_path, parquet_bytes.into()) - .await?; - - let last_checkpoint_content: Value = serde_json::to_value(checkpoint)?; - let last_checkpoint_content = bytes::Bytes::from(serde_json::to_vec(&last_checkpoint_content)?); - - debug!("Writing _last_checkpoint to {last_checkpoint_path:?}."); - object_store - .put(&last_checkpoint_path, last_checkpoint_content.into()) - .await?; - - Ok(()) -} - /// Deletes all delta log commits that are older than the cutoff time /// and less than the specified version. pub async fn cleanup_expired_logs_for( @@ -211,23 +162,20 @@ pub async fn cleanup_expired_logs_for( log_store: &dyn LogStore, cutoff_timestamp: i64, operation_id: Option, -) -> Result { +) -> DeltaResult { static DELTA_LOG_REGEX: LazyLock = LazyLock::new(|| { Regex::new(r"_delta_log/(\d{20})\.(json|checkpoint|json.tmp).*$").unwrap() }); let object_store = log_store.object_store(None); - let maybe_last_checkpoint = object_store - .get(&log_store.log_path().child("_last_checkpoint")) - .await; + let log_path = log_store.log_path(); + let maybe_last_checkpoint = read_last_checkpoint(&object_store, log_path).await?; - if let Err(Error::NotFound { path: _, source: _ }) = maybe_last_checkpoint { + let Some(last_checkpoint) = maybe_last_checkpoint else { return Ok(0); - } + }; - let last_checkpoint = maybe_last_checkpoint?.bytes().await?; - let last_checkpoint: CheckPoint = serde_json::from_slice(&last_checkpoint)?; - let until_version = i64::min(until_version, last_checkpoint.version); + let until_version = i64::min(until_version, last_checkpoint.version as i64); // Feed a stream of candidate deletion files directly into the delete_stream // function to try to improve the speed of cleanup and reduce the need for @@ -270,324 +218,27 @@ pub async fn cleanup_expired_logs_for( Ok(deleted.len()) } -fn parquet_bytes_from_state( - state: &DeltaTableState, - mut tombstones: Vec, -) -> Result<(CheckPoint, bytes::Bytes), ProtocolError> { - let current_metadata = state.metadata(); - let schema = current_metadata.schema()?; - - let partition_col_data_types = get_partition_col_data_types(&schema, current_metadata); - - // Collect a map of paths that require special stats conversion. - let mut stats_conversions: Vec<(SchemaPath, DataType)> = Vec::new(); - let fields = schema.fields().collect_vec(); - collect_stats_conversions(&mut stats_conversions, fields.as_slice()); - - // if any, tombstones do not include extended file metadata, we must omit the extended metadata fields from the remove schema - // See https://github.com/delta-io/delta/blob/master/PROTOCOL.md#add-file-and-remove-file - // - // DBR version 8.x and greater have different behaviors of reading the parquet file depending - // on the `extended_file_metadata` flag, hence this is safer to set `extended_file_metadata=false` - // and omit metadata columns if at least one remove action has `extended_file_metadata=false`. - // We've added the additional check on `size.is_some` because in delta-spark the primitive long type - // is used, hence we want to omit possible errors when `extended_file_metadata=true`, but `size=null` - let use_extended_remove_schema = tombstones - .iter() - .all(|r| r.extended_file_metadata == Some(true) && r.size.is_some()); - - // If use_extended_remove_schema=false for some of the tombstones, then it should be for each. - if !use_extended_remove_schema { - for remove in tombstones.iter_mut() { - remove.extended_file_metadata = Some(false); - } - } - let files = state - .file_actions_iter() - .map_err(|e| ProtocolError::Generic(e.to_string()))?; - // protocol - let jsons = std::iter::once(Action::Protocol(Protocol { - min_reader_version: state.protocol().min_reader_version, - min_writer_version: state.protocol().min_writer_version, - writer_features: if state.protocol().min_writer_version >= 7 { - Some(state.protocol().writer_features.clone().unwrap_or_default()) - } else { - None - }, - reader_features: if state.protocol().min_reader_version >= 3 { - Some(state.protocol().reader_features.clone().unwrap_or_default()) - } else { - None - }, - })) - // metaData - .chain(std::iter::once(Action::Metadata(current_metadata.clone()))) - // txns - .chain( - state - .app_transaction_version() - .map_err(|_| CheckpointError::MissingActionType("txn".to_string()))? - .map(Action::Txn), - ) - // removes - .chain(tombstones.iter().map(|r| { - let mut r = (*r).clone(); - - // As a "new writer", we should always set `extendedFileMetadata` when writing, and include/ignore the other three fields accordingly. - // https://github.com/delta-io/delta/blob/fb0452c2fb142310211c6d3604eefb767bb4a134/core/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala#L311-L314 - if r.extended_file_metadata.is_none() { - r.extended_file_metadata = Some(false); - } - - Action::Remove(r) - })) - .map(|a| serde_json::to_value(a).map_err(ProtocolError::from)) - // adds - .chain(files.map(|f| { - checkpoint_add_from_state( - &f, - partition_col_data_types.as_slice(), - &stats_conversions, - state.table_config().write_stats_as_json(), - state.table_config().write_stats_as_struct(), - ) - })); - - // Create the arrow schema that represents the Checkpoint parquet file. - let arrow_schema = delta_log_schema_for_table( - (&schema).try_into()?, - current_metadata.partition_columns.as_slice(), - use_extended_remove_schema, - state.table_config().write_stats_as_json(), - state.table_config().write_stats_as_struct(), - ); - - debug!("Writing to checkpoint parquet buffer..."); - - let writer_properties = if state.table_config().use_checkpoint_rle() { - WriterProperties::builder() - .set_compression(Compression::SNAPPY) - .build() - } else { - WriterProperties::builder() - .set_compression(Compression::SNAPPY) - .set_dictionary_enabled(false) - .set_encoding(Encoding::PLAIN) - .build() +/// Try reading the `_last_checkpoint` file. +/// +/// Note that we typically want to ignore a missing/invalid `_last_checkpoint` file without failing +/// the read. Thus, the semantics of this function are to return `None` if the file is not found or +/// is invalid JSON. Unexpected/unrecoverable errors are returned as `Err` case and are assumed to +/// cause failure. +async fn read_last_checkpoint( + storage: &dyn ObjectStore, + log_path: &Path, +) -> DeltaResult> { + const LAST_CHECKPOINT_FILE_NAME: &str = "_last_checkpoint"; + let file_path = log_path.child(LAST_CHECKPOINT_FILE_NAME); + let maybe_data = storage.get(&file_path).await; + let data = match maybe_data { + Ok(data) => data.bytes().await?, + Err(Error::NotFound { .. }) => return Ok(None), + Err(err) => return Err(err.into()), }; - - // Write the Checkpoint parquet file. - let mut bytes = vec![]; - let mut writer = - ArrowWriter::try_new(&mut bytes, arrow_schema.clone(), Some(writer_properties))?; - let mut decoder = ReaderBuilder::new(arrow_schema) - .with_batch_size(CHECKPOINT_RECORD_BATCH_SIZE) - .build_decoder()?; - - // Count of actions - let mut total_actions = 0; - - let span = tracing::debug_span!("serialize_checkpoint").entered(); - for chunk in &jsons.chunks(CHECKPOINT_RECORD_BATCH_SIZE) { - let mut buf = Vec::new(); - for j in chunk { - serde_json::to_writer(&mut buf, &j?)?; - total_actions += 1; - } - let _ = decoder.decode(&buf)?; - while let Some(batch) = decoder.flush()? { - writer.write(&batch)?; - } - } - drop(span); - - let _ = writer.close()?; - debug!(total_actions, "Finished writing checkpoint parquet buffer."); - - let checkpoint = CheckPointBuilder::new(state.version(), total_actions) - .with_size_in_bytes(bytes.len() as i64) - .build(); - Ok((checkpoint, bytes::Bytes::from(bytes))) -} - -fn checkpoint_add_from_state( - add: &AddAction, - partition_col_data_types: &[(&String, &DataType)], - stats_conversions: &[(SchemaPath, DataType)], - write_stats_as_json: bool, - write_stats_as_struct: bool, -) -> Result { - let mut v = serde_json::to_value(Action::Add(add.clone())) - .map_err(|err| ArrowError::JsonError(err.to_string()))?; - - v["add"]["dataChange"] = Value::Bool(false); - - // Only created partitionValues_parsed when delta.checkpoint.writeStatsAsStruct is enabled - if !add.partition_values.is_empty() && write_stats_as_struct { - let mut partition_values_parsed: HashMap = HashMap::new(); - - for (field_name, data_type) in partition_col_data_types.iter() { - if let Some(string_value) = add.partition_values.get(*field_name) { - let v = typed_partition_value_from_option_string(string_value, data_type)?; - - partition_values_parsed.insert(field_name.to_string(), v); - } - } - - let partition_values_parsed = serde_json::to_value(partition_values_parsed) - .map_err(|err| ArrowError::JsonError(err.to_string()))?; - v["add"]["partitionValues_parsed"] = partition_values_parsed; - } - - // Only created stats_parsed when delta.checkpoint.writeStatsAsStruct is enabled - if write_stats_as_struct { - if let Ok(Some(stats)) = add.get_stats() { - let mut stats = serde_json::to_value(stats) - .map_err(|err| ArrowError::JsonError(err.to_string()))?; - let min_values = stats.get_mut("minValues").and_then(|v| v.as_object_mut()); - - if let Some(min_values) = min_values { - for (path, data_type) in stats_conversions { - apply_stats_conversion(min_values, path.as_slice(), data_type) - } - } - - let max_values = stats.get_mut("maxValues").and_then(|v| v.as_object_mut()); - if let Some(max_values) = max_values { - for (path, data_type) in stats_conversions { - apply_stats_conversion(max_values, path.as_slice(), data_type) - } - } - - v["add"]["stats_parsed"] = stats; - } - } - - // Don't write stats when delta.checkpoint.writeStatsAsJson is disabled - if !write_stats_as_json { - v.get_mut("add") - .and_then(|v| v.as_object_mut()) - .and_then(|v| v.remove("stats")); - } - Ok(v) -} - -fn typed_partition_value_from_string( - string_value: &str, - data_type: &DataType, -) -> Result { - match data_type { - DataType::Primitive(primitive_type) => match primitive_type { - PrimitiveType::String | PrimitiveType::Binary => Ok(string_value.to_owned().into()), - PrimitiveType::Long - | PrimitiveType::Integer - | PrimitiveType::Short - | PrimitiveType::Byte => Ok(string_value - .parse::() - .map_err(|_| CheckpointError::PartitionValueNotParseable(string_value.to_owned()))? - .into()), - PrimitiveType::Boolean => Ok(string_value - .parse::() - .map_err(|_| CheckpointError::PartitionValueNotParseable(string_value.to_owned()))? - .into()), - PrimitiveType::Float | PrimitiveType::Double => Ok(string_value - .parse::() - .map_err(|_| CheckpointError::PartitionValueNotParseable(string_value.to_owned()))? - .into()), - PrimitiveType::Date => { - let d = NaiveDate::parse_from_str(string_value, "%Y-%m-%d").map_err(|_| { - CheckpointError::PartitionValueNotParseable(string_value.to_owned()) - })?; - // day 0 is 1970-01-01 (719163 days from ce) - Ok((d.num_days_from_ce() - 719_163).into()) - } - PrimitiveType::Timestamp | PrimitiveType::TimestampNtz => { - let ts = NaiveDateTime::parse_from_str(string_value, "%Y-%m-%d %H:%M:%S.%6f"); - let ts: NaiveDateTime = match ts { - Ok(_) => ts, - Err(_) => NaiveDateTime::parse_from_str(string_value, "%Y-%m-%d %H:%M:%S"), - } - .map_err(|_| { - CheckpointError::PartitionValueNotParseable(string_value.to_owned()) - })?; - Ok((ts.and_utc().timestamp_millis() * 1000).into()) - } - s => unimplemented!("Primitive type {s} is not supported for partition column values."), - }, - d => unimplemented!("Data type {d:?} is not supported for partition column values."), - } -} - -fn typed_partition_value_from_option_string( - string_value: &Option, - data_type: &DataType, -) -> Result { - match string_value { - Some(s) => { - if s.is_empty() { - Ok(Value::Null) // empty string should be deserialized as null - } else { - typed_partition_value_from_string(s, data_type) - } - } - None => Ok(Value::Null), - } -} - -fn collect_stats_conversions(paths: &mut Vec<(SchemaPath, DataType)>, fields: &[&StructField]) { - let mut _path = SchemaPath::new(); - fields - .iter() - .for_each(|f| collect_field_conversion(&mut _path, paths, f)); -} - -fn collect_field_conversion( - current_path: &mut SchemaPath, - all_paths: &mut Vec<(SchemaPath, DataType)>, - field: &StructField, -) { - match field.data_type() { - DataType::Primitive(PrimitiveType::Timestamp) => { - let mut key_path = current_path.clone(); - key_path.push(field.name().to_owned()); - all_paths.push((key_path, field.data_type().to_owned())); - } - DataType::Struct(struct_field) => { - let struct_fields = struct_field.fields(); - current_path.push(field.name().to_owned()); - struct_fields.for_each(|f| collect_field_conversion(current_path, all_paths, f)); - current_path.pop(); - } - _ => { /* noop */ } - } -} - -fn apply_stats_conversion( - context: &mut serde_json::Map, - path: &[String], - data_type: &DataType, -) { - if path.len() == 1 { - if let DataType::Primitive(PrimitiveType::Timestamp) = data_type { - let v = context.get_mut(&path[0]); - - if let Some(v) = v { - let ts = v - .as_str() - .and_then(|s| time_utils::timestamp_micros_from_stats_string(s).ok()) - .map(|n| Value::Number(serde_json::Number::from(n))); - - if let Some(ts) = ts { - *v = ts; - } - } - } - } else { - let next_context = context.get_mut(&path[0]).and_then(|v| v.as_object_mut()); - if let Some(next_context) = next_context { - apply_stats_conversion(next_context, &path[1..], data_type); - } - } + Ok(serde_json::from_slice(&data) + .inspect_err(|e| warn!("invalid _last_checkpoint JSON: {e}")) + .ok()) } #[cfg(test)] @@ -599,11 +250,10 @@ mod tests { use arrow_schema::Schema as ArrowSchema; use chrono::Duration; use object_store::path::Path; - use serde_json::json; use super::*; - use crate::kernel::StructType; - use crate::operations::transaction::{CommitBuilder, TableReference}; + use crate::kernel::transaction::{CommitBuilder, TableReference}; + use crate::kernel::Action; use crate::operations::DeltaOps; use crate::protocol::Metadata; use crate::writer::test_utils::get_delta_schema; @@ -619,31 +269,28 @@ mod tests { .with_save_mode(crate::protocol::SaveMode::Ignore) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_schema().unwrap(), &table_schema); - let res = - create_checkpoint_for(0, table.snapshot().unwrap(), table.log_store.as_ref(), None) - .await; + let res = create_checkpoint_for(0, table.log_store.as_ref(), None).await; assert!(res.is_ok()); // Look at the "files" and verify that the _last_checkpoint has the right version - let path = Path::from("_delta_log/_last_checkpoint"); - let last_checkpoint = table - .object_store() - .get(&path) + let log_path = Path::from("_delta_log"); + let store = table.log_store().object_store(None); + let last_checkpoint = read_last_checkpoint(store.as_ref(), &log_path) .await .expect("Failed to get the _last_checkpoint") - .bytes() - .await - .expect("Failed to get bytes for _last_checkpoint"); - let last_checkpoint: CheckPoint = serde_json::from_slice(&last_checkpoint).expect("Fail"); + .expect("Expected checkpoint hint"); assert_eq!(last_checkpoint.version, 0); } /// This test validates that a checkpoint can be written and re-read with the minimum viable /// Metadata. There was a bug which didn't handle the optionality of createdTime. + #[cfg(feature = "datafusion")] #[tokio::test] async fn test_create_checkpoint_with_metadata() { + use std::collections::HashMap; + let table_schema = get_delta_schema(); let mut table = DeltaOps::new_in_memory() @@ -652,7 +299,7 @@ mod tests { .with_save_mode(crate::protocol::SaveMode::Ignore) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_schema().unwrap(), &table_schema); let part_cols: Vec = vec![]; @@ -699,13 +346,12 @@ mod tests { table.load().await.expect("Failed to reload table"); assert_eq!( table.version(), - 1, + Some(1), "The loaded version of the table is not up to date" ); let res = create_checkpoint_for( - table.version(), - table.state.as_ref().unwrap(), + table.version().unwrap() as u64, table.log_store.as_ref(), None, ) @@ -713,22 +359,18 @@ mod tests { assert!(res.is_ok()); // Look at the "files" and verify that the _last_checkpoint has the right version - let path = Path::from("_delta_log/_last_checkpoint"); - let last_checkpoint = table - .object_store() - .get(&path) + let log_path = Path::from("_delta_log"); + let store = table.log_store().object_store(None); + let last_checkpoint = read_last_checkpoint(store.as_ref(), &log_path) .await .expect("Failed to get the _last_checkpoint") - .bytes() - .await - .expect("Failed to get bytes for _last_checkpoint"); - let last_checkpoint: CheckPoint = serde_json::from_slice(&last_checkpoint).expect("Fail"); + .expect("Expected checkpoint hint"); assert_eq!(last_checkpoint.version, 1); // If the regression exists, this will fail table.load().await.expect("Failed to reload the table, this likely means that the optional createdTime was not actually optional"); assert_eq!( - 1, + Some(1), table.version(), "The reloaded table doesn't have the right version" ); @@ -744,11 +386,9 @@ mod tests { .with_save_mode(crate::protocol::SaveMode::Ignore) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_schema().unwrap(), &table_schema); - match create_checkpoint_for(1, table.snapshot().unwrap(), table.log_store.as_ref(), None) - .await - { + match create_checkpoint_for(1, table.log_store.as_ref(), None).await { Ok(_) => { /* * If a checkpoint is allowed to be created here, it will use the passed in @@ -769,134 +409,7 @@ mod tests { } } - #[test] - fn typed_partition_value_from_string_test() { - let string_value: Value = "Hello World!".into(); - assert_eq!( - string_value, - typed_partition_value_from_option_string( - &Some("Hello World!".to_string()), - &DataType::Primitive(PrimitiveType::String), - ) - .unwrap() - ); - - let bool_value: Value = true.into(); - assert_eq!( - bool_value, - typed_partition_value_from_option_string( - &Some("true".to_string()), - &DataType::Primitive(PrimitiveType::Boolean), - ) - .unwrap() - ); - - let number_value: Value = 42.into(); - assert_eq!( - number_value, - typed_partition_value_from_option_string( - &Some("42".to_string()), - &DataType::Primitive(PrimitiveType::Integer), - ) - .unwrap() - ); - - for (s, v) in [ - ("2021-08-08", 18_847), - ("1970-01-02", 1), - ("1970-01-01", 0), - ("1969-12-31", -1), - ("1-01-01", -719_162), - ] { - let date_value: Value = v.into(); - assert_eq!( - date_value, - typed_partition_value_from_option_string( - &Some(s.to_string()), - &DataType::Primitive(PrimitiveType::Date), - ) - .unwrap() - ); - } - - for (s, v) in [ - ("2021-08-08 01:00:01.000000", 1628384401000000i64), - ("2021-08-08 01:00:01", 1628384401000000i64), - ("1970-01-02 12:59:59.000000", 133199000000i64), - ("1970-01-02 12:59:59", 133199000000i64), - ("1970-01-01 13:00:01.000000", 46801000000i64), - ("1970-01-01 13:00:01", 46801000000i64), - ("1969-12-31 00:00:00", -86400000000i64), - ("1677-09-21 00:12:44", -9223372036000000i64), - ] { - let timestamp_value: Value = v.into(); - assert_eq!( - timestamp_value, - typed_partition_value_from_option_string( - &Some(s.to_string()), - &DataType::Primitive(PrimitiveType::Timestamp), - ) - .unwrap() - ); - } - - let binary_value: Value = "\u{2081}\u{2082}\u{2083}\u{2084}".into(); - assert_eq!( - binary_value, - typed_partition_value_from_option_string( - &Some("₁₂₃₄".to_string()), - &DataType::Primitive(PrimitiveType::Binary), - ) - .unwrap() - ); - } - - #[test] - fn null_partition_value_from_string_test() { - assert_eq!( - Value::Null, - typed_partition_value_from_option_string( - &None, - &DataType::Primitive(PrimitiveType::Integer), - ) - .unwrap() - ); - - // empty string should be treated as null - assert_eq!( - Value::Null, - typed_partition_value_from_option_string( - &Some("".to_string()), - &DataType::Primitive(PrimitiveType::Integer), - ) - .unwrap() - ); - } - - #[test] - fn collect_stats_conversions_test() { - let delta_schema: StructType = serde_json::from_value(SCHEMA.clone()).unwrap(); - let fields = delta_schema.fields().collect_vec(); - let mut paths = Vec::new(); - collect_stats_conversions(&mut paths, fields.as_slice()); - - assert_eq!(2, paths.len()); - assert_eq!( - ( - vec!["some_struct".to_string(), "struct_timestamp".to_string()], - DataType::Primitive(PrimitiveType::Timestamp) - ), - paths[0] - ); - assert_eq!( - ( - vec!["some_timestamp".to_string()], - DataType::Primitive(PrimitiveType::Timestamp) - ), - paths[1] - ); - } - + #[cfg(feature = "datafusion")] async fn setup_table() -> DeltaTable { use arrow_schema::{DataType, Field}; let schema = Arc::new(ArrowSchema::new(vec![Field::new( @@ -921,6 +434,7 @@ mod tests { .unwrap() } + #[cfg(feature = "datafusion")] #[tokio::test] async fn test_cleanup_no_checkpoints() { // Test that metadata clean up does not corrupt the table when no checkpoints exist @@ -935,7 +449,7 @@ mod tests { .log_retention_duration() .as_millis() as i64; let count = cleanup_expired_logs_for( - table.version(), + table.version().unwrap(), table.log_store().as_ref(), log_retention_timestamp, None, @@ -950,6 +464,7 @@ mod tests { assert!(res.is_ok()); } + #[cfg(feature = "datafusion")] #[tokio::test] async fn test_cleanup_with_checkpoints() { let table = setup_table().await; @@ -964,7 +479,7 @@ mod tests { .log_retention_duration() .as_millis() as i64; let count = cleanup_expired_logs_for( - table.version(), + table.version().unwrap(), table.log_store().as_ref(), log_retention_timestamp, None, @@ -990,95 +505,7 @@ mod tests { assert!(res.is_ok()); } - #[test] - fn apply_stats_conversion_test() { - let mut stats = STATS_JSON.clone(); - - let min_values = stats.get_mut("minValues").unwrap().as_object_mut().unwrap(); - - apply_stats_conversion( - min_values, - &["some_struct".to_string(), "struct_string".to_string()], - &DataType::Primitive(PrimitiveType::String), - ); - apply_stats_conversion( - min_values, - &["some_struct".to_string(), "struct_timestamp".to_string()], - &DataType::Primitive(PrimitiveType::Timestamp), - ); - apply_stats_conversion( - min_values, - &["some_string".to_string()], - &DataType::Primitive(PrimitiveType::String), - ); - apply_stats_conversion( - min_values, - &["some_timestamp".to_string()], - &DataType::Primitive(PrimitiveType::Timestamp), - ); - - let max_values = stats.get_mut("maxValues").unwrap().as_object_mut().unwrap(); - - apply_stats_conversion( - max_values, - &["some_struct".to_string(), "struct_string".to_string()], - &DataType::Primitive(PrimitiveType::String), - ); - apply_stats_conversion( - max_values, - &["some_struct".to_string(), "struct_timestamp".to_string()], - &DataType::Primitive(PrimitiveType::Timestamp), - ); - apply_stats_conversion( - max_values, - &["some_string".to_string()], - &DataType::Primitive(PrimitiveType::String), - ); - apply_stats_conversion( - max_values, - &["some_timestamp".to_string()], - &DataType::Primitive(PrimitiveType::Timestamp), - ); - - // minValues - assert_eq!( - "A", - stats["minValues"]["some_struct"]["struct_string"] - .as_str() - .unwrap() - ); - assert_eq!( - 1627668684594000i64, - stats["minValues"]["some_struct"]["struct_timestamp"] - .as_i64() - .unwrap() - ); - assert_eq!("P", stats["minValues"]["some_string"].as_str().unwrap()); - assert_eq!( - 1627668684594000i64, - stats["minValues"]["some_timestamp"].as_i64().unwrap() - ); - - // maxValues - assert_eq!( - "B", - stats["maxValues"]["some_struct"]["struct_string"] - .as_str() - .unwrap() - ); - assert_eq!( - 1627668685594000i64, - stats["maxValues"]["some_struct"]["struct_timestamp"] - .as_i64() - .unwrap() - ); - assert_eq!("Q", stats["maxValues"]["some_string"].as_str().unwrap()); - assert_eq!( - 1627668685594000i64, - stats["maxValues"]["some_timestamp"].as_i64().unwrap() - ); - } - + #[cfg(feature = "datafusion")] #[tokio::test] async fn test_struct_with_single_list_field() { // you need another column otherwise the entire stats struct is empty @@ -1116,55 +543,8 @@ mod tests { create_checkpoint(&table, None).await.unwrap(); } - static SCHEMA: LazyLock = LazyLock::new(|| { - json!({ - "type": "struct", - "fields": [ - { - "name": "some_struct", - "type": { - "type": "struct", - "fields": [ - { - "name": "struct_string", - "type": "string", - "nullable": true, "metadata": {} - }, - { - "name": "struct_timestamp", - "type": "timestamp", - "nullable": true, "metadata": {} - }] - }, - "nullable": true, "metadata": {} - }, - { "name": "some_string", "type": "string", "nullable": true, "metadata": {} }, - { "name": "some_timestamp", "type": "timestamp", "nullable": true, "metadata": {} }, - ] - }) - }); - static STATS_JSON: LazyLock = LazyLock::new(|| { - json!({ - "minValues": { - "some_struct": { - "struct_string": "A", - "struct_timestamp": "2021-07-30T18:11:24.594Z" - }, - "some_string": "P", - "some_timestamp": "2021-07-30T18:11:24.594Z" - }, - "maxValues": { - "some_struct": { - "struct_string": "B", - "struct_timestamp": "2021-07-30T18:11:25.594Z" - }, - "some_string": "Q", - "some_timestamp": "2021-07-30T18:11:25.594Z" - } - }) - }); - #[ignore = "This test is only useful if the batch size has been made small"] + #[cfg(feature = "datafusion")] #[tokio::test] async fn test_checkpoint_large_table() -> DeltaResult<()> { use crate::writer::test_utils::get_arrow_schema; @@ -1178,7 +558,7 @@ mod tests { .with_columns(table_schema.fields().cloned()) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let count = 20; for _ in 0..count { @@ -1201,7 +581,11 @@ mod tests { } table.load().await?; - assert_eq!(table.version(), count, "Expected {count} transactions"); + assert_eq!( + table.version().unwrap(), + count, + "Expected {count} transactions" + ); let pre_checkpoint_actions = table.snapshot()?.file_actions()?; let before = table.version(); @@ -1252,7 +636,7 @@ mod tests { .write(vec![batch]) .await?; table.load().await?; - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); create_checkpoint(&table, None).await?; @@ -1271,7 +655,7 @@ mod tests { .write(vec![batch]) .with_save_mode(SaveMode::Overwrite) .await?; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let expected = [ "+----+-------+------------+", diff --git a/crates/core/src/protocol/mod.rs b/crates/core/src/protocol/mod.rs index 2e3e1de801..4252513839 100644 --- a/crates/core/src/protocol/mod.rs +++ b/crates/core/src/protocol/mod.rs @@ -7,98 +7,16 @@ use std::collections::HashMap; use std::hash::{Hash, Hasher}; use std::mem::take; use std::str::FromStr; -use std::sync::LazyLock; -use arrow_schema::ArrowError; -use futures::StreamExt; -use object_store::{path::Path, Error as ObjectStoreError, ObjectStore}; -use regex::Regex; use serde::{Deserialize, Serialize}; use serde_json::Value; -use tracing::{debug, error}; use crate::errors::{DeltaResult, DeltaTableError}; use crate::kernel::{Add, CommitInfo, Metadata, Protocol, Remove, StructField, TableFeatures}; -use crate::logstore::LogStore; -use crate::table::CheckPoint; pub mod checkpoints; -mod parquet_read; -mod time_utils; - -/// Error returned when an invalid Delta log action is encountered. -#[allow(missing_docs)] -#[derive(thiserror::Error, Debug)] -pub enum ProtocolError { - #[error("Table state does not contain metadata")] - NoMetaData, - - #[error("Checkpoint file not found")] - CheckpointNotFound, - - #[error("End of transaction log")] - EndOfLog, - - /// The action contains an invalid field. - #[error("Invalid action field: {0}")] - InvalidField(String), - - /// A parquet log checkpoint file contains an invalid action. - #[error("Invalid action in parquet row: {0}")] - InvalidRow(String), - - /// A transaction log contains invalid deletion vector storage type - #[error("Invalid deletion vector storage type: {0}")] - InvalidDeletionVectorStorageType(String), - - /// A generic action error. The wrapped error string describes the details. - #[error("Generic action error: {0}")] - Generic(String), - - /// Error returned when parsing checkpoint parquet using the parquet crate. - #[error("Failed to parse parquet checkpoint: {source}")] - ParquetParseError { - /// Parquet error details returned when parsing the checkpoint parquet - #[from] - source: parquet::errors::ParquetError, - }, - - /// Failed to serialize operation - #[error("Failed to serialize operation: {source}")] - SerializeOperation { - #[from] - /// The source error - source: serde_json::Error, - }, - - /// Error returned when converting the schema to Arrow format failed. - #[error("Failed to convert into Arrow schema: {}", .source)] - Arrow { - /// Arrow error details returned when converting the schema in Arrow format failed - #[from] - source: ArrowError, - }, - - /// Passthrough error returned when calling ObjectStore. - #[error("ObjectStoreError: {source}")] - ObjectStore { - /// The source ObjectStoreError. - #[from] - source: ObjectStoreError, - }, - - #[error("Io: {source}")] - IO { - #[from] - source: std::io::Error, - }, - #[error("Kernel: {source}")] - Kernel { - #[from] - source: crate::kernel::Error, - }, -} +pub(crate) use checkpoints::{cleanup_expired_logs_for, create_checkpoint_for}; /// Struct used to represent minValues and maxValues in add action statistics. #[derive(Serialize, Deserialize, Debug, PartialEq, Eq)] @@ -244,22 +162,12 @@ impl Eq for Add {} impl Add { /// Get whatever stats are available. Uses (parquet struct) parsed_stats if present falling back to json stats. pub fn get_stats(&self) -> Result, serde_json::error::Error> { - match self.get_stats_parsed() { - Ok(Some(stats)) => Ok(Some(stats)), - Ok(None) => self.get_json_stats(), - Err(e) => { - error!( - "Error when reading parquet stats {:?} {e}. Attempting to read json stats", - self.stats_parsed - ); - self.get_json_stats() - } - } + self.get_json_stats() } /// Returns the serde_json representation of stats contained in the action if present. /// Since stats are defined as optional in the protocol, this may be None. - pub fn get_json_stats(&self) -> Result, serde_json::error::Error> { + fn get_json_stats(&self) -> Result, serde_json::error::Error> { self.stats .as_ref() .map(|stats| serde_json::from_str(stats).map(|mut ps: PartialStats| ps.as_stats())) @@ -456,6 +364,12 @@ pub enum DeltaOperation { /// Fields added to existing schema fields: Vec, }, + /// Update table metadata operations + #[serde(rename_all = "camelCase")] + UpdateTableMetadata { + /// The metadata update to apply + metadata_update: crate::operations::update_table_metadata::TableMetadataUpdate, + }, } impl DeltaOperation { @@ -484,13 +398,13 @@ impl DeltaOperation { DeltaOperation::DropConstraint { .. } => "DROP CONSTRAINT", DeltaOperation::AddFeature { .. } => "ADD FEATURE", DeltaOperation::UpdateFieldMetadata { .. } => "UPDATE FIELD METADATA", + DeltaOperation::UpdateTableMetadata { .. } => "UPDATE TABLE METADATA", } } /// Parameters configured for operation. pub fn operation_parameters(&self) -> DeltaResult> { - if let Some(Some(Some(map))) = serde_json::to_value(self) - .map_err(|err| ProtocolError::SerializeOperation { source: err })? + if let Some(Some(Some(map))) = serde_json::to_value(self)? .as_object() .map(|p| p.values().next().map(|q| q.as_object())) { @@ -509,10 +423,9 @@ impl DeltaOperation { }) .collect()) } else { - Err(ProtocolError::Generic( + Err(DeltaTableError::Generic( "Operation parameters serialized into unexpected shape".into(), - ) - .into()) + )) } } @@ -521,6 +434,7 @@ impl DeltaOperation { match self { Self::Optimize { .. } | Self::UpdateFieldMetadata { .. } + | Self::UpdateTableMetadata { .. } | Self::SetTableProperties { .. } | Self::AddColumn { .. } | Self::AddFeature { .. } @@ -609,83 +523,6 @@ pub enum OutputMode { Update, } -pub(crate) async fn get_last_checkpoint( - log_store: &dyn LogStore, -) -> Result { - let last_checkpoint_path = Path::from_iter(["_delta_log", "_last_checkpoint"]); - debug!("loading checkpoint from {last_checkpoint_path}"); - match log_store - .object_store(None) - .get(&last_checkpoint_path) - .await - { - Ok(data) => Ok(serde_json::from_slice(&data.bytes().await?)?), - Err(ObjectStoreError::NotFound { .. }) => { - match find_latest_check_point_for_version(log_store, i64::MAX).await { - Ok(Some(cp)) => Ok(cp), - _ => Err(ProtocolError::CheckpointNotFound), - } - } - Err(err) => Err(ProtocolError::ObjectStore { source: err }), - } -} - -pub(crate) async fn find_latest_check_point_for_version( - log_store: &dyn LogStore, - version: i64, -) -> Result, ProtocolError> { - static CHECKPOINT_REGEX: LazyLock = - LazyLock::new(|| Regex::new(r"^_delta_log/(\d{20})\.checkpoint\.parquet$").unwrap()); - static CHECKPOINT_PARTS_REGEX: LazyLock = LazyLock::new(|| { - Regex::new(r"^_delta_log/(\d{20})\.checkpoint\.\d{10}\.(\d{10})\.parquet$").unwrap() - }); - - let mut cp: Option = None; - let object_store = log_store.object_store(None); - let mut stream = object_store.list(Some(log_store.log_path())); - - while let Some(obj_meta) = stream.next().await { - // Exit early if any objects can't be listed. - // We exclude the special case of a not found error on some of the list entities. - // This error mainly occurs for local stores when a temporary file has been deleted by - // concurrent writers or if the table is vacuumed by another client. - let obj_meta = match obj_meta { - Ok(meta) => Ok(meta), - Err(ObjectStoreError::NotFound { .. }) => continue, - Err(err) => Err(err), - }?; - if let Some(captures) = CHECKPOINT_REGEX.captures(obj_meta.location.as_ref()) { - let curr_ver_str = captures.get(1).unwrap().as_str(); - let curr_ver: i64 = curr_ver_str.parse().unwrap(); - if curr_ver > version { - // skip checkpoints newer than max version - continue; - } - if cp.is_none() || curr_ver > cp.unwrap().version { - cp = Some(CheckPoint::new(curr_ver, 0, None)); - } - continue; - } - - if let Some(captures) = CHECKPOINT_PARTS_REGEX.captures(obj_meta.location.as_ref()) { - let curr_ver_str = captures.get(1).unwrap().as_str(); - let curr_ver: i64 = curr_ver_str.parse().unwrap(); - if curr_ver > version { - // skip checkpoints newer than max version - continue; - } - if cp.is_none() || curr_ver > cp.unwrap().version { - let parts_str = captures.get(2).unwrap().as_str(); - let parts = parts_str.parse().unwrap(); - cp = Some(CheckPoint::new(curr_ver, 0, Some(parts))); - } - continue; - } - } - - Ok(cp) -} - #[cfg(test)] mod tests { use super::*; @@ -707,7 +544,6 @@ mod tests { data_change: true, deletion_vector: None, partition_values: Default::default(), - stats_parsed: None, tags: None, size: 0, modification_time: 0, @@ -782,7 +618,6 @@ mod tests { data_change: true, deletion_vector: None, partition_values: Default::default(), - stats_parsed: None, tags: None, size: 0, modification_time: 0, @@ -931,6 +766,7 @@ mod tests { } #[tokio::test] + #[ignore = "enable when deletion vector is supported"] async fn test_with_deletion_vector() { // test table with partitions let path = "../test/tests/data/table_with_deletion_logs"; diff --git a/crates/core/src/protocol/parquet_read/mod.rs b/crates/core/src/protocol/parquet_read/mod.rs deleted file mode 100644 index a5c8379467..0000000000 --- a/crates/core/src/protocol/parquet_read/mod.rs +++ /dev/null @@ -1,757 +0,0 @@ -use std::{collections::HashMap, str::FromStr}; - -use chrono::{SecondsFormat, TimeZone, Utc}; -use delta_kernel::table_features::{ReaderFeatures, WriterFeatures}; -use num_bigint::BigInt; -use num_traits::cast::ToPrimitive; -use parquet::record::{Field, ListAccessor, MapAccessor, RowAccessor}; -use serde_json::json; -use tracing::{debug, error, warn}; - -use crate::kernel::models::actions::serde_path::decode_path; -use crate::kernel::{ - Action, Add, AddCDCFile, DeletionVectorDescriptor, Metadata, Protocol, Remove, StorageType, - Transaction, -}; -use crate::protocol::{ColumnCountStat, ColumnValueStat, ProtocolError, Stats}; - -fn populate_hashmap_with_option_from_parquet_map( - map: &mut HashMap>, - pmap: &parquet::record::Map, -) -> Result<(), &'static str> { - let keys = pmap.get_keys(); - let values = pmap.get_values(); - for j in 0..pmap.len() { - map.entry( - keys.get_string(j) - .map_err(|_| "key for HashMap in parquet has to be a string")? - .clone(), - ) - .or_insert_with(|| values.get_string(j).ok().cloned()); - } - - Ok(()) -} - -fn gen_action_type_error(action: &str, field: &str, expected_type: &str) -> ProtocolError { - ProtocolError::InvalidField(format!( - "type for {field} in {action} action should be {expected_type}" - )) -} - -impl AddCDCFile { - fn from_parquet_record(_record: &parquet::record::Row) -> Result { - let re = Self { - ..Default::default() - }; - Ok(re) - } -} - -impl DeletionVectorDescriptor { - fn from_parquet_record(record: &parquet::record::Row) -> Result { - let mut re = Self { - cardinality: -1, - offset: None, - path_or_inline_dv: "".to_string(), - size_in_bytes: -1, - storage_type: StorageType::default(), - }; - for (i, (name, _)) in record.get_column_iter().enumerate() { - match name.as_str() { - "storageType" => { - re.storage_type = - StorageType::from_str(record.get_string(i).map_err(|_| { - gen_action_type_error("add", "deletionVector.storage_type", "string") - })?)?; - } - "pathOrInlineDv" => { - re.path_or_inline_dv = record - .get_string(i) - .map_err(|_| { - gen_action_type_error("add", "deletionVector.pathOrInlineDv", "string") - })? - .clone(); - } - "offset" => { - re.offset = match record.get_int(i) { - Ok(x) => Some(x), - _ => None, - } - } - "sizeInBytes" => { - re.size_in_bytes = record.get_int(i).map_err(|_| { - gen_action_type_error("add", "deletionVector.sizeInBytes", "int") - })?; - } - "cardinality" => { - re.cardinality = record.get_long(i).map_err(|_| { - gen_action_type_error("add", "deletionVector.sizeInBytes", "long") - })?; - } - _ => { - debug!("Unexpected field name `{name}` for deletion vector: {record:?}"); - } - } - } - Ok(re) - } -} - -impl Add { - fn from_parquet_record(record: &parquet::record::Row) -> Result { - let mut re = Self { - path: "".to_string(), - size: -1, - modification_time: -1, - data_change: true, - partition_values: HashMap::new(), - stats: None, - stats_parsed: None, - deletion_vector: None, - base_row_id: None, - default_row_commit_version: None, - tags: None, - clustering_provider: None, - }; - - for (i, (name, _)) in record.get_column_iter().enumerate() { - match name.as_str() { - "path" => { - re.path = decode_path( - record - .get_string(i) - .map_err(|_| gen_action_type_error("add", "path", "string"))? - .clone() - .as_str(), - )?; - } - "size" => { - re.size = record - .get_long(i) - .map_err(|_| gen_action_type_error("add", "size", "long"))?; - } - "modificationTime" => { - re.modification_time = record - .get_long(i) - .map_err(|_| gen_action_type_error("add", "modificationTime", "long"))?; - } - "dataChange" => { - re.data_change = record - .get_bool(i) - .map_err(|_| gen_action_type_error("add", "dataChange", "bool"))?; - } - "partitionValues" => { - let parquet_map = record - .get_map(i) - .map_err(|_| gen_action_type_error("add", "partitionValues", "map"))?; - populate_hashmap_with_option_from_parquet_map( - &mut re.partition_values, - parquet_map, - ) - .map_err(|estr| { - ProtocolError::InvalidField(format!( - "Invalid partitionValues for add action: {estr}", - )) - })?; - } - // "partitionValues_parsed" => { - // re.partition_values_parsed = Some( - // record - // .get_group(i) - // .map_err(|_| { - // gen_action_type_error("add", "partitionValues_parsed", "struct") - // })? - // .clone(), - // ); - // } - "tags" => match record.get_map(i) { - Ok(tags_map) => { - let mut tags = HashMap::new(); - populate_hashmap_with_option_from_parquet_map(&mut tags, tags_map) - .map_err(|estr| { - ProtocolError::InvalidField(format!( - "Invalid tags for add action: {estr}", - )) - })?; - re.tags = Some(tags); - } - _ => { - re.tags = None; - } - }, - "stats" => match record.get_string(i) { - Ok(stats) => { - re.stats = Some(stats.clone()); - } - _ => { - re.stats = None; - } - }, - "stats_parsed" => match record.get_group(i) { - Ok(stats_parsed) => { - re.stats_parsed = Some(stats_parsed.clone()); - } - _ => { - re.stats_parsed = None; - } - }, - "deletionVector" => match record.get_group(i) { - Ok(row) => { - re.deletion_vector = - Some(DeletionVectorDescriptor::from_parquet_record(row)?); - } - _ => { - re.deletion_vector = None; - } - }, - _ => { - debug!("Unexpected field name `{name}` for add action: {record:?}"); - } - } - } - - Ok(re) - } - - /// Returns the composite HashMap representation of stats contained in the action if present. - /// Since stats are defined as optional in the protocol, this may be None. - pub fn get_stats_parsed(&self) -> Result, ProtocolError> { - self.stats_parsed.as_ref().map_or(Ok(None), |record| { - let mut stats = Stats::default(); - - for (i, (name, _)) in record.get_column_iter().enumerate() { - match name.as_str() { - "numRecords" => if let Ok(v) = record.get_long(i) { - stats.num_records = v; - } else { - error!("Expect type of stats_parsed field numRecords to be long, got: {record}"); - } - "minValues" => if let Ok(row) = record.get_group(i) { - for (name, field) in row.get_column_iter() { - if !matches!(field, Field::Null) { - if let Some(values) = field_to_value_stat(field, name) { - stats.min_values.insert(name.clone(), values); - } - } - } - } else { - error!("Expect type of stats_parsed field minRecords to be struct, got: {record}"); - } - "maxValues" => if let Ok(row) = record.get_group(i) { - for (name, field) in row.get_column_iter() { - if !matches!(field, Field::Null) { - if let Some(values) = field_to_value_stat(field, name) { - stats.max_values.insert(name.clone(), values); - } - } - } - } else { - error!("Expect type of stats_parsed field maxRecords to be struct, got: {record}"); - } - "nullCount" => if let Ok(row) = record.get_group(i) { - for (name, field) in row.get_column_iter() { - if !matches!(field, Field::Null) { - if let Some(count) = field_to_count_stat(field, name) { - stats.null_count.insert(name.clone(), count); - } - } - } - } else { - error!("Expect type of stats_parsed field nullCount to be struct, got: {record}"); - } - _ => { - debug!("Unexpected field name `{name}` for stats_parsed: {record:?}"); - } - } - } - - Ok(Some(stats)) - }) - } -} - -fn field_to_value_stat(field: &Field, field_name: &str) -> Option { - match field { - Field::Group(group) => { - let values = group.get_column_iter().filter_map(|(name, sub_field)| { - field_to_value_stat(sub_field, name).map(|val| (name.clone(), val)) - }); - Some(ColumnValueStat::Column(HashMap::from_iter(values))) - } - _ => { - if let Ok(val) = primitive_parquet_field_to_json_value(field) { - Some(ColumnValueStat::Value(val)) - } else { - warn!( - "Unexpected type when parsing min/max values for {field_name}. Found {field}" - ); - None - } - } - } -} - -fn field_to_count_stat(field: &Field, field_name: &str) -> Option { - match field { - Field::Group(group) => { - let counts = group.get_column_iter().filter_map(|(name, sub_field)| { - field_to_count_stat(sub_field, name).map(|count| (name.clone(), count)) - }); - Some(ColumnCountStat::Column(HashMap::from_iter(counts))) - } - Field::Long(value) => Some(ColumnCountStat::Value(*value)), - _ => { - warn!("Unexpected type when parsing nullCounts for {field_name}. Found {field}"); - None - } - } -} - -fn primitive_parquet_field_to_json_value(field: &Field) -> Result { - match field { - Field::Bool(value) => Ok(json!(value)), - Field::Byte(value) => Ok(json!(value)), - Field::Short(value) => Ok(json!(value)), - Field::Int(value) => Ok(json!(value)), - Field::Long(value) => Ok(json!(value)), - Field::Float(value) => Ok(json!(value)), - Field::Double(value) => Ok(json!(value)), - Field::Str(value) => Ok(json!(value)), - Field::Decimal(decimal) => match BigInt::from_signed_bytes_be(decimal.data()).to_f64() { - Some(int) => Ok(json!( - int / (10_i64.pow((decimal.scale()).try_into().unwrap()) as f64) - )), - _ => Err("Invalid type for min/max values."), - }, - Field::TimestampMicros(timestamp) => Ok(serde_json::Value::String( - convert_timestamp_micros_to_string(*timestamp)?, - )), - Field::TimestampMillis(timestamp) => Ok(serde_json::Value::String( - convert_timestamp_millis_to_string(*timestamp)?, - )), - Field::Date(date) => Ok(serde_json::Value::String(convert_date_to_string(*date)?)), - _ => Err("Invalid type for min/max values."), - } -} - -fn convert_timestamp_millis_to_string(value: i64) -> Result { - let seconds = value / 1000; - let milliseconds = (value % 1000) as u32; - - let dt = Utc - .timestamp_opt(seconds, milliseconds * 1_000_000) - .single() - .ok_or("Value out of bounds")?; - Ok(dt.to_rfc3339_opts(SecondsFormat::Millis, true)) -} - -fn convert_timestamp_micros_to_string(value: i64) -> Result { - let seconds = value / 1_000_000; - let microseconds = (value % 1_000_000) as u32; - - let dt: chrono::DateTime = Utc - .timestamp_opt(seconds, microseconds * 1_000) - .single() - .ok_or("Value out of bounds")?; - Ok(dt.to_rfc3339_opts(SecondsFormat::Micros, true)) -} - -fn convert_date_to_string(value: i32) -> Result { - static NUM_SECONDS_IN_DAY: i64 = 60 * 60 * 24; - let dt = Utc - .timestamp_opt(value as i64 * NUM_SECONDS_IN_DAY, 0) - .single() - .ok_or("Value out of bounds")? - .date_naive(); - Ok(format!("{}", dt.format("%Y-%m-%d"))) -} - -impl Metadata { - fn from_parquet_record(record: &parquet::record::Row) -> Result { - let mut re = Self { - id: "".to_string(), - name: None, - description: None, - partition_columns: vec![], - schema_string: "".to_string(), - created_time: None, - configuration: HashMap::new(), - format: Default::default(), - }; - - for (i, (name, _)) in record.get_column_iter().enumerate() { - match name.as_str() { - "id" => { - re.id = record - .get_string(i) - .map_err(|_| gen_action_type_error("metaData", "id", "string"))? - .clone(); - } - "name" => match record.get_string(i) { - Ok(s) => re.name = Some(s.clone()), - _ => re.name = None, - }, - "description" => match record.get_string(i) { - Ok(s) => re.description = Some(s.clone()), - _ => re.description = None, - }, - "partitionColumns" => { - let columns_list = record.get_list(i).map_err(|_| { - gen_action_type_error("metaData", "partitionColumns", "list") - })?; - for j in 0..columns_list.len() { - re.partition_columns.push( - columns_list - .get_string(j) - .map_err(|_| { - gen_action_type_error( - "metaData", - "partitionColumns.value", - "string", - ) - })? - .clone(), - ); - } - } - "schemaString" => { - re.schema_string = record - .get_string(i) - .map_err(|_| gen_action_type_error("metaData", "schemaString", "string"))? - .clone(); - } - "createdTime" => match record.get_long(i) { - Ok(s) => re.created_time = Some(s), - _ => re.created_time = None, - }, - "configuration" => { - let configuration_map = record - .get_map(i) - .map_err(|_| gen_action_type_error("metaData", "configuration", "map"))?; - populate_hashmap_with_option_from_parquet_map( - &mut re.configuration, - configuration_map, - ) - .map_err(|estr| { - ProtocolError::InvalidField(format!( - "Invalid configuration for metaData action: {estr}", - )) - })?; - } - "format" => { - let format_record = record - .get_group(i) - .map_err(|_| gen_action_type_error("metaData", "format", "struct"))?; - - re.format.provider = format_record - .get_string(0) - .map_err(|_| { - gen_action_type_error("metaData", "format.provider", "string") - })? - .clone(); - match record.get_map(1) { - Ok(options_map) => { - let mut options = HashMap::new(); - populate_hashmap_with_option_from_parquet_map( - &mut options, - options_map, - ) - .map_err(|estr| { - ProtocolError::InvalidField(format!( - "Invalid format.options for metaData action: {estr}", - )) - })?; - re.format.options = options; - } - _ => { - re.format.options = HashMap::new(); - } - } - } - _ => { - debug!("Unexpected field name `{name}` for metaData action: {record:?}"); - } - } - } - - Ok(re) - } -} - -impl Remove { - fn from_parquet_record(record: &parquet::record::Row) -> Result { - let mut re = Self { - data_change: true, - extended_file_metadata: Some(false), - deletion_timestamp: None, - deletion_vector: None, - partition_values: None, - path: "".to_string(), - size: None, - tags: None, - base_row_id: None, - default_row_commit_version: None, - }; - - for (i, (name, _)) in record.get_column_iter().enumerate() { - match name.as_str() { - "path" => { - re.path = decode_path( - record - .get_string(i) - .map_err(|_| gen_action_type_error("remove", "path", "string"))? - .clone() - .as_str(), - )?; - } - "dataChange" => { - re.data_change = record - .get_bool(i) - .map_err(|_| gen_action_type_error("remove", "dataChange", "bool"))?; - } - "extendedFileMetadata" => { - re.extended_file_metadata = record.get_bool(i).map(Some).unwrap_or(None); - } - "deletionTimestamp" => { - re.deletion_timestamp = Some(record.get_long(i).map_err(|_| { - gen_action_type_error("remove", "deletionTimestamp", "long") - })?); - } - "partitionValues" => match record.get_map(i) { - Ok(_) => { - let parquet_map = record.get_map(i).map_err(|_| { - gen_action_type_error("remove", "partitionValues", "map") - })?; - let mut partition_values = HashMap::new(); - populate_hashmap_with_option_from_parquet_map( - &mut partition_values, - parquet_map, - ) - .map_err(|estr| { - ProtocolError::InvalidField(format!( - "Invalid partitionValues for remove action: {estr}", - )) - })?; - re.partition_values = Some(partition_values); - } - _ => re.partition_values = None, - }, - "tags" => match record.get_map(i) { - Ok(tags_map) => { - let mut tags = HashMap::new(); - populate_hashmap_with_option_from_parquet_map(&mut tags, tags_map) - .map_err(|estr| { - ProtocolError::InvalidField(format!( - "Invalid tags for remove action: {estr}", - )) - })?; - re.tags = Some(tags); - } - _ => { - re.tags = None; - } - }, - "size" => { - re.size = record.get_long(i).map(Some).unwrap_or(None); - } - "numRecords" => {} - _ => { - debug!("Unexpected field name `{name}` for remove action: {record:?}"); - } - } - } - - Ok(re) - } -} - -impl Transaction { - fn from_parquet_record(record: &parquet::record::Row) -> Result { - let mut re = Self { - ..Default::default() - }; - - for (i, (name, _)) in record.get_column_iter().enumerate() { - match name.as_str() { - "appId" => { - re.app_id = record - .get_string(i) - .map_err(|_| gen_action_type_error("txn", "appId", "string"))? - .clone(); - } - "version" => { - re.version = record - .get_long(i) - .map_err(|_| gen_action_type_error("txn", "version", "long"))?; - } - "lastUpdated" => { - re.last_updated = record.get_long(i).map(Some).unwrap_or(None); - } - _ => { - debug!("Unexpected field name `{name}` for txn action: {record:?}"); - } - } - } - - Ok(re) - } -} - -impl Protocol { - fn from_parquet_record(record: &parquet::record::Row) -> Result { - let mut re = Self { - min_reader_version: -1, - min_writer_version: -1, - reader_features: None, - writer_features: None, - }; - - for (i, (name, _)) in record.get_column_iter().enumerate() { - match name.as_str() { - "minReaderVersion" => { - re.min_reader_version = record.get_int(i).map_err(|_| { - gen_action_type_error("protocol", "minReaderVersion", "int") - })?; - } - "minWriterVersion" => { - re.min_writer_version = record.get_int(i).map_err(|_| { - gen_action_type_error("protocol", "minWriterVersion", "int") - })?; - } - "readerFeatures" => { - re.reader_features = record - .get_list(i) - .map(|l| { - l.elements() - .iter() - .filter_map(|v| match v { - Field::Str(feature) => { - ReaderFeatures::try_from(feature.as_str()).ok() - } - _ => None, - }) - .collect() - }) - .ok() - } - "writerFeatures" => { - re.writer_features = record - .get_list(i) - .map(|l| { - l.elements() - .iter() - .filter_map(|v| match v { - Field::Str(feature) => { - WriterFeatures::try_from(feature.as_str()).ok() - } - _ => None, - }) - .collect() - }) - .ok() - } - _ => { - debug!("Unexpected field name `{name}` for protocol action: {record:?}"); - } - } - } - - Ok(re) - } -} - -impl Action { - /// Returns an action from the given parquet Row. Used when deserializing delta log parquet - /// checkpoints. - pub fn from_parquet_record( - schema: &parquet::schema::types::Type, - record: &parquet::record::Row, - ) -> Result { - // find column that's not none - let (col_idx, col_data) = { - let mut col_idx = None; - let mut col_data = None; - for i in 0..record.len() { - match record.get_group(i) { - Ok(group) => { - col_idx = Some(i); - col_data = Some(group); - } - _ => { - continue; - } - } - } - - match (col_idx, col_data) { - (Some(idx), Some(group)) => (idx, group), - _ => { - return Err(ProtocolError::InvalidRow( - "Parquet action row only contains null columns".to_string(), - )); - } - } - }; - - let fields = schema.get_fields(); - let field = &fields[col_idx]; - - Ok(match field.get_basic_info().name() { - "add" => Action::Add(Add::from_parquet_record(col_data)?), - "metaData" => Action::Metadata(Metadata::from_parquet_record(col_data)?), - "remove" => Action::Remove(Remove::from_parquet_record(col_data)?), - "txn" => Action::Txn(Transaction::from_parquet_record(col_data)?), - "protocol" => Action::Protocol(Protocol::from_parquet_record(col_data)?), - "cdc" => Action::Cdc(AddCDCFile::from_parquet_record(col_data)?), - name => { - return Err(ProtocolError::InvalidField(format!( - "Unexpected action from checkpoint: {name}", - ))); - } - }) - } -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_add_action_without_partition_values_and_stats() { - use parquet::file::reader::{FileReader, SerializedFileReader}; - use std::fs::File; - - let path = - "../test/tests/data/delta-0.2.0/_delta_log/00000000000000000003.checkpoint.parquet"; - let preader = SerializedFileReader::new(File::open(path).unwrap()).unwrap(); - - let mut iter = preader.get_row_iter(None).unwrap(); - let record = iter.nth(9).unwrap().unwrap(); - let add_record = record.get_group(1).unwrap(); - let add_action = Add::from_parquet_record(add_record).unwrap(); - - assert_eq!(add_action.partition_values.len(), 0); - assert_eq!(add_action.stats, None); - } - - #[test] - fn test_issue_1372_field_to_value_stat() { - let now = Utc::now(); - let timestamp_milliseconds = Field::TimestampMillis(now.timestamp_millis()); - let ts_millis = field_to_value_stat(×tamp_milliseconds, "timestamp_millis"); - assert!(ts_millis.is_some()); - assert_eq!( - ColumnValueStat::Value(json!(now.to_rfc3339_opts(SecondsFormat::Millis, true))), - ts_millis.unwrap() - ); - - let timestamp_milliseconds = Field::TimestampMicros(now.timestamp_micros()); - let ts_micros = field_to_value_stat(×tamp_milliseconds, "timestamp_micros"); - assert!(ts_micros.is_some()); - assert_eq!( - ColumnValueStat::Value(json!(now.to_rfc3339_opts(SecondsFormat::Micros, true))), - ts_micros.unwrap() - ); - } -} diff --git a/crates/core/src/protocol/time_utils.rs b/crates/core/src/protocol/time_utils.rs deleted file mode 100644 index 72465ea7a5..0000000000 --- a/crates/core/src/protocol/time_utils.rs +++ /dev/null @@ -1,54 +0,0 @@ -//! Utility functions for converting time formats. -#![allow(unused)] - -use arrow::temporal_conversions; -use parquet::basic::TimeUnit; - -/// Convert an ISO-8601/RFC3339 timestamp string to a numeric microsecond epoch representation. -/// Stats strings are written with millisecond precision as described by the delta protocol. -pub fn timestamp_micros_from_stats_string(s: &str) -> Result { - chrono::DateTime::parse_from_rfc3339(s).map(|dt| dt.timestamp_millis() * 1000) -} - -/// Convert the timestamp to a ISO-8601 style format suitable for JSON statistics. -pub fn timestamp_to_delta_stats_string(n: i64, time_unit: &TimeUnit) -> Option { - let dt = match time_unit { - TimeUnit::MILLIS(_) => temporal_conversions::timestamp_ms_to_datetime(n), - TimeUnit::MICROS(_) => temporal_conversions::timestamp_us_to_datetime(n), - TimeUnit::NANOS(_) => temporal_conversions::timestamp_ns_to_datetime(n), - }?; - - Some(format!("{}", dt.format("%Y-%m-%dT%H:%M:%S%.3fZ"))) -} - -#[cfg(test)] -mod tests { - use super::*; - use parquet::format::{MicroSeconds, MilliSeconds, NanoSeconds, TimeUnit}; - - #[test] - fn test_timestamp_to_delta_stats_string() { - let s = - timestamp_to_delta_stats_string(1628685199541, &TimeUnit::MILLIS(MilliSeconds::new())) - .unwrap(); - assert_eq!("2021-08-11T12:33:19.541Z".to_string(), s); - let s = timestamp_to_delta_stats_string( - 1628685199541000, - &TimeUnit::MICROS(MicroSeconds::new()), - ) - .unwrap(); - assert_eq!("2021-08-11T12:33:19.541Z".to_string(), s); - let s = timestamp_to_delta_stats_string( - 1628685199541000000, - &TimeUnit::NANOS(NanoSeconds::new()), - ) - .unwrap(); - assert_eq!("2021-08-11T12:33:19.541Z".to_string(), s); - } - - #[test] - fn test_timestamp_micros_from_stats_string() { - let us = timestamp_micros_from_stats_string("2021-08-11T12:33:19.541Z").unwrap(); - assert_eq!(1628685199541000i64, us); - } -} diff --git a/crates/core/src/schema/partitions.rs b/crates/core/src/schema/partitions.rs index c55a065340..35660148d0 100644 --- a/crates/core/src/schema/partitions.rs +++ b/crates/core/src/schema/partitions.rs @@ -53,15 +53,17 @@ impl PartialOrd for ScalarHelper<'_> { (TimestampNtz(a), TimestampNtz(b)) => a.partial_cmp(b), (Date(a), Date(b)) => a.partial_cmp(b), (Binary(a), Binary(b)) => a.partial_cmp(b), - (Decimal(a, p1, s1), Decimal(b, p2, s2)) => { + (Decimal(decimal1), Decimal(decimal2)) => { // TODO implement proper decimal comparison - if p1 != p2 || s1 != s2 { + if decimal1.precision() != decimal2.precision() + || decimal1.scale() != decimal2.scale() + { return None; }; - a.partial_cmp(b) + decimal1.bits().partial_cmp(&decimal2.bits()) } // TODO should we make an assumption about the ordering of nulls? - // rigth now this is only used for internal purposes. + // right now this is only used for internal purposes. (Null(_), _) => Some(Ordering::Less), (_, Null(_)) => Some(Ordering::Greater), _ => None, @@ -292,11 +294,11 @@ impl TryFrom<&str> for DeltaTablePartition { /// Try to create a DeltaTable partition from a HivePartition string. /// Returns a DeltaTableError if the string is not in the form of a HivePartition. fn try_from(partition: &str) -> Result { - let partition_splitted: Vec<&str> = partition.split('=').collect(); - match partition_splitted { - partition_splitted if partition_splitted.len() == 2 => Ok(DeltaTablePartition { - key: partition_splitted[0].to_owned(), - value: Scalar::String(partition_splitted[1].to_owned()), + let partition_split: Vec<&str> = partition.split('=').collect(); + match partition_split { + partition_split if partition_split.len() == 2 => Ok(DeltaTablePartition { + key: partition_split[0].to_owned(), + value: Scalar::String(partition_split[1].to_owned()), }), _ => Err(DeltaTableError::PartitionError { partition: partition.to_string(), diff --git a/crates/core/src/storage/file.rs b/crates/core/src/storage/file.rs deleted file mode 100644 index 100faafcc5..0000000000 --- a/crates/core/src/storage/file.rs +++ /dev/null @@ -1,441 +0,0 @@ -//! Local file storage backend. This backend read and write objects from local filesystem. -//! -//! The local file storage backend is multi-writer safe. -use std::ops::Range; -use std::sync::Arc; - -use bytes::Bytes; -use futures::stream::BoxStream; -use object_store::{ - local::LocalFileSystem, path::Path as ObjectStorePath, Error as ObjectStoreError, GetOptions, - GetResult, ListResult, ObjectMeta, ObjectStore, PutOptions, PutResult, - Result as ObjectStoreResult, -}; -use object_store::{MultipartUpload, PutMultipartOpts, PutPayload}; -use url::Url; - -const STORE_NAME: &str = "DeltaLocalObjectStore"; - -/// Error raised by storage lock client -#[derive(thiserror::Error, Debug)] -#[allow(dead_code)] -pub enum LocalFileSystemError { - /// Object exists already at path - #[error("Object exists already at path: {} ({:?})", path, source)] - AlreadyExists { - /// Path of the already existing file - path: String, - /// Originating error - source: Box, - }, - - /// Object not found at the given path - #[error("Object not found at path: {} ({:?})", path, source)] - NotFound { - /// Provided path which does not exist - path: String, - /// Originating error - source: Box, - }, - - /// Invalid argument sent to OS call - #[error("Invalid argument in OS call for path: {} ({:?})", path, source)] - InvalidArgument { - /// Provided path - path: String, - /// Originating error - source: errno::Errno, - }, - - /// Null error for path for FFI - #[error("Null error in FFI for path: {} ({:?})", path, source)] - NullError { - /// Given path - path: String, - /// Originating error - source: std::ffi::NulError, - }, - - /// Generic catch-all error for this store - #[error("Generic error in store: {} ({:?})", store, source)] - Generic { - /// String name of the object store - store: &'static str, - /// Originating error - source: Box, - }, - - /// Errors from the Tokio runtime - #[error("Error executing async task for path: {} ({:?})", path, source)] - Tokio { - /// Path - path: String, - /// Originating error - source: tokio::task::JoinError, - }, -} - -impl From for ObjectStoreError { - fn from(e: LocalFileSystemError) -> Self { - match e { - LocalFileSystemError::AlreadyExists { path, source } => { - ObjectStoreError::AlreadyExists { path, source } - } - LocalFileSystemError::NotFound { path, source } => { - ObjectStoreError::NotFound { path, source } - } - LocalFileSystemError::InvalidArgument { source, .. } => ObjectStoreError::Generic { - store: STORE_NAME, - source: Box::new(source), - }, - LocalFileSystemError::NullError { source, .. } => ObjectStoreError::Generic { - store: STORE_NAME, - source: Box::new(source), - }, - LocalFileSystemError::Tokio { source, .. } => ObjectStoreError::Generic { - store: STORE_NAME, - source: Box::new(source), - }, - LocalFileSystemError::Generic { store, source } => { - ObjectStoreError::Generic { store, source } - } - } - } -} - -/// Multi-writer support for different platforms: -/// -/// * Modern Linux kernels are well supported. However because Linux implementation leverages -/// `RENAME_NOREPLACE`, older versions of the kernel might not work depending on what filesystem is -/// being used: -/// * ext4 requires >= Linux 3.15 -/// * btrfs, shmem, and cif requires >= Linux 3.17 -/// * xfs requires >= Linux 4.0 -/// * ext2, minix, reiserfs, jfs, vfat, and bpf requires >= Linux 4.9 -/// * Darwin is supported but not fully tested. -/// Patches welcome. -/// * Support for other platforms are not implemented at the moment. -#[derive(Debug)] -pub struct FileStorageBackend { - inner: Arc, - root_url: Arc, -} - -impl FileStorageBackend { - /// Creates a new FileStorageBackend. - pub fn try_new(path: impl AsRef) -> ObjectStoreResult { - Ok(Self { - root_url: Arc::new(Self::path_to_root_url(path.as_ref())?), - inner: Arc::new(LocalFileSystem::new_with_prefix(path)?), - }) - } - - fn path_to_root_url(path: &std::path::Path) -> ObjectStoreResult { - let root_path = - std::fs::canonicalize(path).map_err(|e| object_store::Error::InvalidPath { - source: object_store::path::Error::Canonicalize { - path: path.into(), - source: e, - }, - })?; - - Url::from_file_path(root_path).map_err(|_| object_store::Error::InvalidPath { - source: object_store::path::Error::InvalidPath { path: path.into() }, - }) - } - - /// Return an absolute filesystem path of the given location - fn path_to_filesystem(&self, location: &ObjectStorePath) -> String { - let mut url = self.root_url.as_ref().clone(); - url.path_segments_mut() - .expect("url path") - // technically not necessary as Path ignores empty segments - // but avoids creating paths with "//" which look odd in error messages. - .pop_if_empty() - .extend(location.parts()); - - url.to_file_path().unwrap().to_str().unwrap().to_owned() - } -} - -impl std::fmt::Display for FileStorageBackend { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "FileStorageBackend") - } -} - -#[async_trait::async_trait] -impl ObjectStore for FileStorageBackend { - async fn put( - &self, - location: &ObjectStorePath, - bytes: PutPayload, - ) -> ObjectStoreResult { - self.inner.put(location, bytes).await - } - - async fn put_opts( - &self, - location: &ObjectStorePath, - bytes: PutPayload, - options: PutOptions, - ) -> ObjectStoreResult { - self.inner.put_opts(location, bytes, options).await - } - - async fn get(&self, location: &ObjectStorePath) -> ObjectStoreResult { - self.inner.get(location).await - } - - async fn get_opts( - &self, - location: &ObjectStorePath, - options: GetOptions, - ) -> ObjectStoreResult { - self.inner.get_opts(location, options).await - } - - async fn get_range( - &self, - location: &ObjectStorePath, - range: Range, - ) -> ObjectStoreResult { - self.inner.get_range(location, range).await - } - - async fn head(&self, location: &ObjectStorePath) -> ObjectStoreResult { - self.inner.head(location).await - } - - async fn delete(&self, location: &ObjectStorePath) -> ObjectStoreResult<()> { - self.inner.delete(location).await - } - - fn list( - &self, - prefix: Option<&ObjectStorePath>, - ) -> BoxStream<'_, ObjectStoreResult> { - self.inner.list(prefix) - } - - fn list_with_offset( - &self, - prefix: Option<&ObjectStorePath>, - offset: &ObjectStorePath, - ) -> BoxStream<'_, ObjectStoreResult> { - self.inner.list_with_offset(prefix, offset) - } - - async fn list_with_delimiter( - &self, - prefix: Option<&ObjectStorePath>, - ) -> ObjectStoreResult { - self.inner.list_with_delimiter(prefix).await - } - - async fn copy(&self, from: &ObjectStorePath, to: &ObjectStorePath) -> ObjectStoreResult<()> { - self.inner.copy(from, to).await - } - - async fn copy_if_not_exists( - &self, - from: &ObjectStorePath, - to: &ObjectStorePath, - ) -> ObjectStoreResult<()> { - self.inner.copy_if_not_exists(from, to).await - } - - async fn rename_if_not_exists( - &self, - from: &ObjectStorePath, - to: &ObjectStorePath, - ) -> ObjectStoreResult<()> { - let path_from = self.path_to_filesystem(from); - let path_to = self.path_to_filesystem(to); - Ok(rename_noreplace(path_from.as_ref(), path_to.as_ref()).await?) - } - - async fn put_multipart( - &self, - location: &ObjectStorePath, - ) -> ObjectStoreResult> { - self.inner.put_multipart(location).await - } - - async fn put_multipart_opts( - &self, - location: &ObjectStorePath, - options: PutMultipartOpts, - ) -> ObjectStoreResult> { - self.inner.put_multipart_opts(location, options).await - } -} - -/// Atomically renames `from` to `to`. -/// `from` has to exist, but `to` is not, otherwise the operation will fail. -#[inline] -async fn rename_noreplace(from: &str, to: &str) -> Result<(), LocalFileSystemError> { - imp::rename_noreplace(from, to).await -} - -// Generic implementation (Requires 2 system calls) -#[cfg(not(any(all(target_os = "linux", target_env = "gnu"), target_os = "macos")))] -mod imp { - use super::*; - - pub(super) async fn rename_noreplace(from: &str, to: &str) -> Result<(), LocalFileSystemError> { - let from_path = String::from(from); - let to_path = String::from(to); - - tokio::task::spawn_blocking(move || { - std::fs::hard_link(&from_path, &to_path).map_err(|err| { - if err.kind() == std::io::ErrorKind::AlreadyExists { - LocalFileSystemError::AlreadyExists { - path: to_path, - source: Box::new(err), - } - } else if err.kind() == std::io::ErrorKind::NotFound { - LocalFileSystemError::NotFound { - path: from_path.clone(), - source: Box::new(err), - } - } else { - LocalFileSystemError::Generic { - store: STORE_NAME, - source: Box::new(err), - } - } - })?; - - std::fs::remove_file(from_path).map_err(|err| LocalFileSystemError::Generic { - store: STORE_NAME, - source: Box::new(err), - })?; - - Ok(()) - }) - .await - .unwrap() - } -} - -// Optimized implementations (Only 1 system call) -#[cfg(any(all(target_os = "linux", target_env = "gnu"), target_os = "macos"))] -mod imp { - use super::*; - use std::ffi::CString; - - fn to_c_string(p: &str) -> Result { - CString::new(p).map_err(|err| LocalFileSystemError::NullError { - path: p.into(), - source: err, - }) - } - - pub(super) async fn rename_noreplace(from: &str, to: &str) -> Result<(), LocalFileSystemError> { - let cs_from = to_c_string(from)?; - let cs_to = to_c_string(to)?; - - let ret = unsafe { - tokio::task::spawn_blocking(move || { - let ret = platform_specific_rename(cs_from.as_ptr(), cs_to.as_ptr()); - if ret != 0 { - Err(errno::errno()) - } else { - Ok(()) - } - }) - .await - .map_err(|err| LocalFileSystemError::Tokio { - path: from.into(), - source: err, - })? - }; - - match ret { - Err(e) if e.0 == libc::EEXIST => Err(LocalFileSystemError::AlreadyExists { - path: to.into(), - source: Box::new(e), - }), - Err(e) if e.0 == libc::ENOENT => Err(LocalFileSystemError::NotFound { - path: to.into(), - source: Box::new(e), - }), - Err(e) if e.0 == libc::EINVAL => Err(LocalFileSystemError::InvalidArgument { - path: to.into(), - source: e, - }), - Err(e) => Err(LocalFileSystemError::Generic { - store: STORE_NAME, - source: Box::new(e), - }), - Ok(_) => Ok(()), - } - } - - #[allow(unused_variables)] - unsafe fn platform_specific_rename(from: *const libc::c_char, to: *const libc::c_char) -> i32 { - cfg_if::cfg_if! { - if #[cfg(all(target_os = "linux", target_env = "gnu"))] { - libc::renameat2(libc::AT_FDCWD, from, libc::AT_FDCWD, to, libc::RENAME_NOREPLACE) - } else if #[cfg(target_os = "macos")] { - libc::renamex_np(from, to, libc::RENAME_EXCL) - } else { - unreachable!() - } - } - } -} - -#[cfg(test)] -mod tests { - use super::*; - use std::fs::File; - use std::io::Write; - use std::path::{Path, PathBuf}; - - #[tokio::test] - async fn test_rename_noreplace() { - let tmp_dir = tempfile::tempdir().unwrap(); - let a = create_file(tmp_dir.path(), "a"); - let b = create_file(tmp_dir.path(), "b"); - let c = &tmp_dir.path().join("c"); - - // unsuccessful move not_exists to C, not_exists is missing - let result = rename_noreplace("not_exists", c.to_str().unwrap()).await; - assert!(matches!( - result.expect_err("nonexistent should fail"), - LocalFileSystemError::NotFound { .. } - )); - - // successful move A to C - assert!(a.exists()); - assert!(!c.exists()); - match rename_noreplace(a.to_str().unwrap(), c.to_str().unwrap()).await { - Err(LocalFileSystemError::InvalidArgument {source, ..}) => - panic!("expected success, got: {source:?}. Note: atomically renaming Windows files from WSL2 is not supported."), - Err(e) => panic!("expected success, got: {e:?}"), - _ => {} - } - assert!(!a.exists()); - assert!(c.exists()); - - // unsuccessful move B to C, C already exists, B is not deleted - assert!(b.exists()); - match rename_noreplace(b.to_str().unwrap(), c.to_str().unwrap()).await { - Err(LocalFileSystemError::AlreadyExists { path, .. }) => { - assert_eq!(path, c.to_str().unwrap()) - } - _ => panic!("unexpected"), - } - assert!(b.exists()); - assert_eq!(std::fs::read_to_string(c).unwrap(), "a"); - } - - fn create_file(dir: &Path, name: &str) -> PathBuf { - let path = dir.join(name); - let mut file = File::create(&path).unwrap(); - file.write_all(name.as_bytes()).unwrap(); - path - } -} diff --git a/crates/core/src/storage/mod.rs b/crates/core/src/storage/mod.rs deleted file mode 100644 index d850a8822c..0000000000 --- a/crates/core/src/storage/mod.rs +++ /dev/null @@ -1,674 +0,0 @@ -//! Object storage backend abstraction layer for Delta Table transaction logs and data -use std::collections::HashMap; -use std::sync::{Arc, LazyLock, OnceLock}; - -use crate::{DeltaResult, DeltaTableError}; -use dashmap::DashMap; -use futures::future::BoxFuture; -use futures::FutureExt; -use futures::TryFutureExt; -use humantime::parse_duration; -use object_store::limit::LimitStore; -use object_store::local::LocalFileSystem; -use object_store::memory::InMemory; -use object_store::prefix::PrefixStore; -#[cfg(feature = "cloud")] -use object_store::RetryConfig; -use object_store::{GetOptions, PutOptions, PutPayload, PutResult}; -use serde::{Deserialize, Serialize}; -use tokio::runtime::{Builder as RuntimeBuilder, Handle, Runtime}; -use url::Url; - -use bytes::Bytes; -use futures::stream::BoxStream; -pub use object_store; -pub use object_store::path::{Path, DELIMITER}; -pub use object_store::{ - DynObjectStore, Error as ObjectStoreError, GetResult, ListResult, MultipartId, ObjectMeta, - ObjectStore, Result as ObjectStoreResult, -}; -use object_store::{MultipartUpload, PutMultipartOpts}; -pub use retry_ext::ObjectStoreRetryExt; -use std::ops::Range; -pub use utils::*; - -pub mod file; -pub mod retry_ext; -pub mod utils; - -static DELTA_LOG_PATH: LazyLock = LazyLock::new(|| Path::from("_delta_log")); - -/// Creates static IO Runtime with optional configuration -fn io_rt(config: Option<&RuntimeConfig>) -> &Runtime { - static IO_RT: OnceLock = OnceLock::new(); - IO_RT.get_or_init(|| { - let rt = match config { - Some(config) => { - let mut builder = if config.multi_threaded { - RuntimeBuilder::new_multi_thread() - } else { - RuntimeBuilder::new_current_thread() - }; - let builder = builder.worker_threads(config.worker_threads); - #[allow(unused_mut)] - let mut builder = if config.enable_io && config.enable_time { - builder.enable_all() - } else if !config.enable_io && config.enable_time { - builder.enable_time() - } else { - builder - }; - #[cfg(unix)] - { - if config.enable_io && !config.enable_time { - builder = builder.enable_io(); - } - } - builder - .thread_name( - config - .thread_name - .clone() - .unwrap_or("IO-runtime".to_string()), - ) - .build() - } - _ => Runtime::new(), - }; - rt.expect("Failed to create a tokio runtime for IO.") - }) -} - -/// Configuration for Tokio runtime -#[derive(Debug, Clone, Serialize, Deserialize)] -pub struct RuntimeConfig { - multi_threaded: bool, - worker_threads: usize, - thread_name: Option, - enable_io: bool, - enable_time: bool, -} - -/// Provide custom Tokio RT or a runtime config -#[derive(Debug, Clone)] -pub enum IORuntime { - /// Tokio RT handle - RT(Handle), - /// Configuration for tokio runtime - Config(RuntimeConfig), -} - -impl Default for IORuntime { - fn default() -> Self { - IORuntime::RT(io_rt(None).handle().clone()) - } -} - -impl IORuntime { - /// Retrieves the Tokio runtime for IO bound operations - pub fn get_handle(&self) -> Handle { - match self { - IORuntime::RT(handle) => handle, - IORuntime::Config(config) => io_rt(Some(config)).handle(), - } - .clone() - } -} - -/// Wraps any object store and runs IO in it's own runtime [EXPERIMENTAL] -pub struct DeltaIOStorageBackend { - inner: ObjectStoreRef, - rt_handle: Handle, -} - -impl DeltaIOStorageBackend { - /// create wrapped object store which spawns tasks in own runtime - pub fn new(storage: ObjectStoreRef, rt_handle: Handle) -> Self { - Self { - inner: storage, - rt_handle, - } - } - - /// spawn tasks on IO runtime - pub fn spawn_io_rt( - &self, - f: F, - store: &Arc, - path: Path, - ) -> BoxFuture<'_, ObjectStoreResult> - where - F: for<'a> FnOnce( - &'a Arc, - &'a Path, - ) -> BoxFuture<'a, ObjectStoreResult> - + Send - + 'static, - O: Send + 'static, - { - let store = Arc::clone(store); - let fut = self.rt_handle.spawn(async move { f(&store, &path).await }); - fut.unwrap_or_else(|e| match e.try_into_panic() { - Ok(p) => std::panic::resume_unwind(p), - Err(e) => Err(ObjectStoreError::JoinError { source: e }), - }) - .boxed() - } - - /// spawn tasks on IO runtime - pub fn spawn_io_rt_from_to( - &self, - f: F, - store: &Arc, - from: Path, - to: Path, - ) -> BoxFuture<'_, ObjectStoreResult> - where - F: for<'a> FnOnce( - &'a Arc, - &'a Path, - &'a Path, - ) -> BoxFuture<'a, ObjectStoreResult> - + Send - + 'static, - O: Send + 'static, - { - let store = Arc::clone(store); - let fut = self - .rt_handle - .spawn(async move { f(&store, &from, &to).await }); - fut.unwrap_or_else(|e| match e.try_into_panic() { - Ok(p) => std::panic::resume_unwind(p), - Err(e) => Err(ObjectStoreError::JoinError { source: e }), - }) - .boxed() - } -} - -impl std::fmt::Debug for DeltaIOStorageBackend { - fn fmt(&self, fmt: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { - write!(fmt, "DeltaIOStorageBackend") - } -} - -impl std::fmt::Display for DeltaIOStorageBackend { - fn fmt(&self, fmt: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { - write!(fmt, "DeltaIOStorageBackend") - } -} - -#[async_trait::async_trait] -impl ObjectStore for DeltaIOStorageBackend { - async fn put(&self, location: &Path, bytes: PutPayload) -> ObjectStoreResult { - self.spawn_io_rt( - |store, path| store.put(path, bytes), - &self.inner, - location.clone(), - ) - .await - } - - async fn put_opts( - &self, - location: &Path, - bytes: PutPayload, - options: PutOptions, - ) -> ObjectStoreResult { - self.spawn_io_rt( - |store, path| store.put_opts(path, bytes, options), - &self.inner, - location.clone(), - ) - .await - } - - async fn get(&self, location: &Path) -> ObjectStoreResult { - self.spawn_io_rt(|store, path| store.get(path), &self.inner, location.clone()) - .await - } - - async fn get_opts(&self, location: &Path, options: GetOptions) -> ObjectStoreResult { - self.spawn_io_rt( - |store, path| store.get_opts(path, options), - &self.inner, - location.clone(), - ) - .await - } - - async fn get_range(&self, location: &Path, range: Range) -> ObjectStoreResult { - self.spawn_io_rt( - |store, path| store.get_range(path, range), - &self.inner, - location.clone(), - ) - .await - } - - async fn head(&self, location: &Path) -> ObjectStoreResult { - self.spawn_io_rt( - |store, path| store.head(path), - &self.inner, - location.clone(), - ) - .await - } - - async fn delete(&self, location: &Path) -> ObjectStoreResult<()> { - self.spawn_io_rt( - |store, path| store.delete(path), - &self.inner, - location.clone(), - ) - .await - } - - fn list(&self, prefix: Option<&Path>) -> BoxStream<'_, ObjectStoreResult> { - self.inner.list(prefix) - } - - fn list_with_offset( - &self, - prefix: Option<&Path>, - offset: &Path, - ) -> BoxStream<'_, ObjectStoreResult> { - self.inner.list_with_offset(prefix, offset) - } - - async fn list_with_delimiter(&self, prefix: Option<&Path>) -> ObjectStoreResult { - self.inner.list_with_delimiter(prefix).await - } - - async fn copy(&self, from: &Path, to: &Path) -> ObjectStoreResult<()> { - self.spawn_io_rt_from_to( - |store, from_path, to_path| store.copy(from_path, to_path), - &self.inner, - from.clone(), - to.clone(), - ) - .await - } - - async fn copy_if_not_exists(&self, from: &Path, to: &Path) -> ObjectStoreResult<()> { - self.spawn_io_rt_from_to( - |store, from_path, to_path| store.copy_if_not_exists(from_path, to_path), - &self.inner, - from.clone(), - to.clone(), - ) - .await - } - - async fn rename_if_not_exists(&self, from: &Path, to: &Path) -> ObjectStoreResult<()> { - self.spawn_io_rt_from_to( - |store, from_path, to_path| store.rename_if_not_exists(from_path, to_path), - &self.inner, - from.clone(), - to.clone(), - ) - .await - } - - async fn put_multipart(&self, location: &Path) -> ObjectStoreResult> { - self.spawn_io_rt( - |store, path| store.put_multipart(path), - &self.inner, - location.clone(), - ) - .await - } - - async fn put_multipart_opts( - &self, - location: &Path, - options: PutMultipartOpts, - ) -> ObjectStoreResult> { - self.spawn_io_rt( - |store, path| store.put_multipart_opts(path, options), - &self.inner, - location.clone(), - ) - .await - } -} - -/// Sharable reference to [`ObjectStore`] -pub type ObjectStoreRef = Arc; - -/// Factory trait for creating [ObjectStoreRef] instances at runtime -pub trait ObjectStoreFactory: Send + Sync { - #[allow(missing_docs)] - fn parse_url_opts( - &self, - url: &Url, - options: &StorageOptions, - ) -> DeltaResult<(ObjectStoreRef, Path)>; -} - -#[cfg(feature = "cloud")] -pub trait RetryConfigParse { - fn parse_retry_config(&self, options: &StorageOptions) -> DeltaResult { - let mut retry_config = RetryConfig::default(); - if let Some(max_retries) = options.0.get("max_retries") { - retry_config.max_retries = max_retries - .parse::() - .map_err(|e| DeltaTableError::generic(e.to_string()))?; - } - - if let Some(retry_timeout) = options.0.get("retry_timeout") { - retry_config.retry_timeout = parse_duration(retry_timeout).map_err(|_| { - DeltaTableError::generic(format!("failed to parse \"{retry_timeout}\" as Duration")) - })?; - } - - if let Some(bc_init_backoff) = options.0.get("backoff_config.init_backoff") { - retry_config.backoff.init_backoff = parse_duration(bc_init_backoff).map_err(|_| { - DeltaTableError::generic(format!( - "failed to parse \"{bc_init_backoff}\" as Duration" - )) - })?; - } - - if let Some(bc_max_backoff) = options.0.get("backoff_config.max_backoff") { - retry_config.backoff.max_backoff = parse_duration(bc_max_backoff).map_err(|_| { - DeltaTableError::generic(format!( - "failed to parse \"{bc_max_backoff}\" as Duration" - )) - })?; - } - - if let Some(bc_base) = options.0.get("backoff_config.base") { - retry_config.backoff.base = bc_base - .parse::() - .map_err(|e| DeltaTableError::generic(e.to_string()))?; - } - - Ok(retry_config) - } -} - -#[derive(Clone, Debug, Default)] -pub(crate) struct DefaultObjectStoreFactory {} - -impl ObjectStoreFactory for DefaultObjectStoreFactory { - fn parse_url_opts( - &self, - url: &Url, - options: &StorageOptions, - ) -> DeltaResult<(ObjectStoreRef, Path)> { - match url.scheme() { - "memory" => { - let path = Path::from_url_path(url.path())?; - let inner = Arc::new(InMemory::new()) as ObjectStoreRef; - let store = limit_store_handler(url_prefix_handler(inner, path.clone()), options); - Ok((store, path)) - } - "file" => { - let inner = Arc::new(LocalFileSystem::new_with_prefix( - url.to_file_path().unwrap(), - )?) as ObjectStoreRef; - let store = limit_store_handler(inner, options); - Ok((store, Path::from("/"))) - } - _ => Err(DeltaTableError::InvalidTableLocation(url.clone().into())), - } - } -} - -pub trait ObjectStoreRegistry: Send + Sync + std::fmt::Debug + 'static + Clone { - /// If a store with the same key existed before, it is replaced and returned - fn register_store( - &self, - url: &Url, - store: Arc, - ) -> Option>; - - /// Get a suitable store for the provided URL. For example: - /// If no [`ObjectStore`] found for the `url`, ad-hoc discovery may be executed depending on - /// the `url` and [`ObjectStoreRegistry`] implementation. An [`ObjectStore`] may be lazily - /// created and registered. - fn get_store(&self, url: &Url) -> DeltaResult>; - - fn all_stores(&self) -> &DashMap>; -} - -/// The default [`ObjectStoreRegistry`] -#[derive(Clone)] -pub struct DefaultObjectStoreRegistry { - /// A map from scheme to object store that serve list / read operations for the store - object_stores: DashMap>, -} - -impl Default for DefaultObjectStoreRegistry { - fn default() -> Self { - Self::new() - } -} - -impl DefaultObjectStoreRegistry { - pub fn new() -> Self { - let object_stores: DashMap> = DashMap::new(); - Self { object_stores } - } -} - -impl std::fmt::Debug for DefaultObjectStoreRegistry { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - f.debug_struct("DefaultObjectStoreRegistry") - .field( - "schemes", - &self - .object_stores - .iter() - .map(|o| o.key().clone()) - .collect::>(), - ) - .finish() - } -} - -impl ObjectStoreRegistry for DefaultObjectStoreRegistry { - fn register_store( - &self, - url: &Url, - store: Arc, - ) -> Option> { - self.object_stores.insert(url.to_string(), store) - } - - fn get_store(&self, url: &Url) -> DeltaResult> { - self.object_stores - .get(&url.to_string()) - .map(|o| Arc::clone(o.value())) - .ok_or_else(|| { - DeltaTableError::generic(format!( - "No suitable object store found for {url}. See `RuntimeEnv::register_object_store`" - )) - }) - } - - fn all_stores(&self) -> &DashMap> { - &self.object_stores - } -} - -/// TODO -pub type FactoryRegistry = Arc>>; - -/// TODO -pub fn factories() -> FactoryRegistry { - static REGISTRY: OnceLock = OnceLock::new(); - REGISTRY - .get_or_init(|| { - let registry = FactoryRegistry::default(); - registry.insert( - Url::parse("memory://").unwrap(), - Arc::new(DefaultObjectStoreFactory::default()), - ); - registry.insert( - Url::parse("file://").unwrap(), - Arc::new(DefaultObjectStoreFactory::default()), - ); - registry - }) - .clone() -} - -/// Simpler access pattern for the [FactoryRegistry] to get a single store -pub fn store_for(url: &Url, storage_options: &StorageOptions) -> DeltaResult { - let scheme = Url::parse(&format!("{}://", url.scheme())).unwrap(); - if let Some(factory) = factories().get(&scheme) { - let (store, _prefix) = factory.parse_url_opts(url, storage_options)?; - Ok(store) - } else { - Err(DeltaTableError::InvalidTableLocation(url.clone().into())) - } -} - -/// Options used for configuring backend storage -#[derive(Clone, Debug, Serialize, Deserialize, Default)] -pub struct StorageOptions(pub HashMap); - -impl From> for StorageOptions { - fn from(value: HashMap) -> Self { - Self(value) - } -} - -/// Return the uri of commit version. -/// -/// ```rust -/// # use deltalake_core::storage::*; -/// use object_store::path::Path; -/// let uri = commit_uri_from_version(1); -/// assert_eq!(uri, Path::from("_delta_log/00000000000000000001.json")); -/// ``` -pub fn commit_uri_from_version(version: i64) -> Path { - let version = format!("{version:020}.json"); - DELTA_LOG_PATH.child(version.as_str()) -} - -/// Return true for all the stringly values typically associated with true -/// -/// aka YAML booleans -/// -/// ```rust -/// # use deltalake_core::storage::*; -/// for value in ["1", "true", "on", "YES", "Y"] { -/// assert!(str_is_truthy(value)); -/// } -/// for value in ["0", "FALSE", "off", "NO", "n", "bork"] { -/// assert!(!str_is_truthy(value)); -/// } -/// ``` -pub fn str_is_truthy(val: &str) -> bool { - val.eq_ignore_ascii_case("1") - | val.eq_ignore_ascii_case("true") - | val.eq_ignore_ascii_case("on") - | val.eq_ignore_ascii_case("yes") - | val.eq_ignore_ascii_case("y") -} - -/// Simple function to wrap the given [ObjectStore] in a [PrefixStore] if necessary -/// -/// This simplifies the use of the storage since it ensures that list/get/etc operations -/// start from the prefix in the object storage rather than from the root configured URI of the -/// [ObjectStore] -pub fn url_prefix_handler(store: T, prefix: Path) -> ObjectStoreRef { - if prefix != Path::from("/") { - Arc::new(PrefixStore::new(store, prefix)) - } else { - Arc::new(store) - } -} - -/// Simple function to wrap the given [ObjectStore] in a [LimitStore] if configured -/// -/// Limits the number of concurrent connections the underlying object store -/// Reference [LimitStore](https://docs.rs/object_store/latest/object_store/limit/struct.LimitStore.html) for more information -pub fn limit_store_handler(store: T, options: &StorageOptions) -> ObjectStoreRef { - let concurrency_limit = options - .0 - .get(storage_constants::OBJECT_STORE_CONCURRENCY_LIMIT) - .and_then(|v| v.parse().ok()); - - if let Some(limit) = concurrency_limit { - Arc::new(LimitStore::new(store, limit)) - } else { - Arc::new(store) - } -} - -/// Storage option keys to use when creating [ObjectStore]. -/// -/// The same key should be used whether passing a key in the hashmap or setting it as an environment variable. -/// Must be implemented for a given storage provider -pub mod storage_constants { - - /// The number of concurrent connections the underlying object store can create - /// Reference [LimitStore](https://docs.rs/object_store/latest/object_store/limit/struct.LimitStore.html) for more information - pub const OBJECT_STORE_CONCURRENCY_LIMIT: &str = "OBJECT_STORE_CONCURRENCY_LIMIT"; -} - -#[cfg(test)] -mod tests { - use std::time::Duration; - - use maplit::hashmap; - - use super::*; - - #[test] - fn test_url_prefix_handler() { - let store = InMemory::new(); - let path = Path::parse("/databases/foo/bar").expect("Failed to parse path"); - - let prefixed = url_prefix_handler(store, path.clone()); - - assert_eq!( - String::from("PrefixObjectStore(databases/foo/bar)"), - format!("{prefixed}") - ); - } - - #[test] - fn test_limit_store_handler() { - let store = InMemory::new(); - - let options = StorageOptions(HashMap::from_iter(vec![( - "OBJECT_STORE_CONCURRENCY_LIMIT".into(), - "500".into(), - )])); - - let limited = limit_store_handler(store, &options); - - assert_eq!( - String::from("LimitStore(500, InMemory)"), - format!("{limited}") - ); - } - - #[cfg(feature = "cloud")] - #[test] - fn test_retry_config_from_options() { - struct TestFactory {} - impl RetryConfigParse for TestFactory {} - - let options = hashmap! { - "max_retries".to_string() => "100".to_string() , - "retry_timeout".to_string() => "300s".to_string() , - "backoff_config.init_backoff".to_string() => "20s".to_string() , - "backoff_config.max_backoff".to_string() => "1h".to_string() , - "backoff_config.base".to_string() => "50.0".to_string() , - }; - let retry_config = TestFactory {} - .parse_retry_config(&StorageOptions(options)) - .unwrap(); - - assert_eq!(retry_config.max_retries, 100); - assert_eq!(retry_config.retry_timeout, Duration::from_secs(300)); - assert_eq!(retry_config.backoff.init_backoff, Duration::from_secs(20)); - assert_eq!(retry_config.backoff.max_backoff, Duration::from_secs(3600)); - assert_eq!(retry_config.backoff.base, 50_f64); - } -} diff --git a/crates/core/src/table/builder.rs b/crates/core/src/table/builder.rs index 9de258b627..4ce5c6dddb 100644 --- a/crates/core/src/table/builder.rs +++ b/crates/core/src/table/builder.rs @@ -5,38 +5,15 @@ use std::path::PathBuf; use std::sync::Arc; use chrono::{DateTime, FixedOffset, Utc}; +use deltalake_derive::DeltaConfig; use object_store::DynObjectStore; use serde::{Deserialize, Serialize}; use tracing::debug; use url::Url; -use super::DeltaTable; -use crate::errors::{DeltaResult, DeltaTableError}; -use crate::logstore::LogStoreRef; -use crate::storage::{factories, IORuntime, StorageOptions}; - -#[allow(dead_code)] -#[derive(Debug, thiserror::Error)] -enum BuilderError { - #[error("Store {backend} requires host in storage url, got: {url}")] - MissingHost { backend: String, url: String }, - #[error("Missing configuration {0}")] - Required(String), - #[error("Failed to find valid credential.")] - MissingCredential, - #[error("Failed to decode SAS key: {0}\nSAS keys must be percent-encoded. They come encoded in the Azure portal and Azure Storage Explorer.")] - Decode(String), - #[error("Delta-rs must be build with feature '{feature}' to support url: {url}.")] - MissingFeature { feature: &'static str, url: String }, - #[error("Failed to parse table uri")] - TableUri(#[from] url::ParseError), -} - -impl From for DeltaTableError { - fn from(err: BuilderError) -> Self { - DeltaTableError::Generic(err.to_string()) - } -} +use crate::logstore::storage::IORuntime; +use crate::logstore::{object_store_factories, LogStoreRef, StorageConfig}; +use crate::{DeltaResult, DeltaTable, DeltaTableError}; /// possible version specifications for loading a delta table #[derive(Debug, Copy, Clone, PartialEq, Eq, Default)] @@ -51,23 +28,16 @@ pub enum DeltaVersion { } /// Configuration options for delta table -#[derive(Debug, Serialize, Deserialize, Clone)] +#[derive(Debug, Serialize, Deserialize, Clone, DeltaConfig)] #[serde(rename_all = "camelCase")] pub struct DeltaTableConfig { - /// Indicates whether our use case requires tracking tombstones. - /// This defaults to `true` - /// - /// Read-only applications never require tombstones. Tombstones - /// are only required when writing checkpoints, so even many writers - /// may want to skip them. - pub require_tombstones: bool, - /// Indicates whether DeltaTable should track files. /// This defaults to `true` /// /// Some append-only applications might have no need of tracking any files. /// Hence, DeltaTable will be loaded with significant memory reduction. pub require_files: bool, + /// Controls how many files to buffer from the commit log when updating the table. /// This defaults to 4 * number of cpus /// @@ -76,10 +46,13 @@ pub struct DeltaTableConfig { /// last checkpoint, but will also increase memory usage. Possible rate limits of the storage backend should /// also be considered for optimal performance. pub log_buffer_size: usize, + /// Control the number of records to read / process from the commit / checkpoint files /// when processing record batches. pub log_batch_size: usize, + #[serde(skip_serializing, skip_deserializing)] + #[delta(skip)] /// When a runtime handler is provided, all IO tasks are spawn in that handle pub io_runtime: Option, } @@ -87,7 +60,6 @@ pub struct DeltaTableConfig { impl Default for DeltaTableConfig { fn default() -> Self { Self { - require_tombstones: true, require_files: true, log_buffer_size: num_cpus::get() * 4, log_batch_size: 1024, @@ -98,8 +70,7 @@ impl Default for DeltaTableConfig { impl PartialEq for DeltaTableConfig { fn eq(&self, other: &Self) -> bool { - self.require_tombstones == other.require_tombstones - && self.require_files == other.require_files + self.require_files == other.require_files && self.log_buffer_size == other.log_buffer_size && self.log_batch_size == other.log_batch_size } @@ -168,12 +139,6 @@ impl DeltaTableBuilder { }) } - /// Sets `require_tombstones=false` to the builder - pub fn without_tombstones(mut self) -> Self { - self.table_config.require_tombstones = false; - self - } - /// Sets `require_files=false` to the builder pub fn without_files(mut self) -> Self { self.table_config.require_files = false; @@ -217,10 +182,15 @@ impl DeltaTableBuilder { /// /// # Arguments /// - /// * `storage` - A shared reference to an [`ObjectStore`](object_store::ObjectStore) with "/" pointing at delta table root (i.e. where `_delta_log` is located). - /// * `location` - A url corresponding to the storagle location of `storage`. - pub fn with_storage_backend(mut self, storage: Arc, location: Url) -> Self { - self.storage_backend = Some((storage, location)); + /// * `root_storage` - A shared reference to an [`ObjectStore`](object_store::ObjectStore) with + /// "/" pointing at the root of the object store. + /// * `location` - A url corresponding to the storage location of the delta table. + pub fn with_storage_backend( + mut self, + root_storage: Arc, + location: Url, + ) -> Self { + self.storage_backend = Some((root_storage, location)); self } @@ -271,7 +241,7 @@ impl DeltaTableBuilder { } /// Storage options for configuring backend object store - pub fn storage_options(&self) -> StorageOptions { + pub fn storage_options(&self) -> HashMap { let mut storage_options = self.storage_options.clone().unwrap_or_default(); if let Some(allow) = self.allow_http { storage_options.insert( @@ -279,7 +249,7 @@ impl DeltaTableBuilder { if allow { "true" } else { "false" }.into(), ); }; - storage_options.into() + storage_options } /// Build a delta storage backend for the given config @@ -289,22 +259,18 @@ impl DeltaTableBuilder { DeltaTableError::NotATable(format!("Could not turn {} into a URL", self.table_uri)) })?; + let mut storage_config = StorageConfig::parse_options(self.storage_options())?; + if let Some(io_runtime) = self.table_config.io_runtime.clone() { + storage_config = storage_config.with_io_runtime(io_runtime); + } + if let Some((store, _url)) = self.storage_backend.as_ref() { debug!("Loading a logstore with a custom store: {store:?}"); - crate::logstore::logstore_with( - store.clone(), - location, - self.storage_options(), - self.table_config.io_runtime.clone(), - ) + crate::logstore::logstore_with(store.clone(), location, storage_config) } else { // If there has been no backend defined just default to the normal logstore look up debug!("Loading a logstore based off the location: {location:?}"); - crate::logstore::logstore_for( - location, - self.storage_options(), - self.table_config.io_runtime.clone(), - ) + crate::logstore::logstore_for(location, storage_config) } } @@ -340,7 +306,7 @@ enum UriType { /// Will return an error if the path is not valid. fn resolve_uri_type(table_uri: impl AsRef) -> DeltaResult { let table_uri = table_uri.as_ref(); - let known_schemes: Vec<_> = factories() + let known_schemes: Vec<_> = object_store_factories() .iter() .map(|v| v.key().scheme().to_owned()) .collect(); @@ -432,11 +398,11 @@ fn ensure_file_location_exists(path: PathBuf) -> DeltaResult<()> { #[cfg(test)] mod tests { use super::*; - use crate::storage::DefaultObjectStoreFactory; + use crate::logstore::factories::DefaultObjectStoreFactory; #[test] fn test_ensure_table_uri() { - factories().insert( + object_store_factories().insert( Url::parse("s3://").unwrap(), Arc::new(DefaultObjectStoreFactory::default()), ); @@ -597,7 +563,7 @@ mod tests { let table = DeltaTableBuilder::from_uri(table_uri).with_storage_options(storage_opts); let found_opts = table.storage_options(); - assert_eq!(expected, found_opts.0.get(key).unwrap()); + assert_eq!(expected, found_opts.get(key).unwrap()); } } } diff --git a/crates/core/src/table/columns.rs b/crates/core/src/table/columns.rs new file mode 100644 index 0000000000..b43b6f24e2 --- /dev/null +++ b/crates/core/src/table/columns.rs @@ -0,0 +1,80 @@ +//! Constraints and generated column mappings +use crate::kernel::DataType; +use crate::table::DataCheck; +use std::any::Any; + +/// A constraint in a check constraint +#[derive(Eq, PartialEq, Debug, Default, Clone)] +pub struct Constraint { + /// The full path to the field. + pub name: String, + /// The SQL string that must always evaluate to true. + pub expr: String, +} + +impl Constraint { + /// Create a new invariant + pub fn new(field_name: &str, invariant_sql: &str) -> Self { + Self { + name: field_name.to_string(), + expr: invariant_sql.to_string(), + } + } +} + +impl DataCheck for Constraint { + fn get_name(&self) -> &str { + &self.name + } + + fn get_expression(&self) -> &str { + &self.expr + } + + fn as_any(&self) -> &dyn Any { + self + } +} + +/// A generated column +#[derive(Eq, PartialEq, Debug, Clone)] +pub struct GeneratedColumn { + /// The full path to the field. + pub name: String, + /// The SQL string that generate the column value. + pub generation_expr: String, + /// The SQL string that must always evaluate to true. + pub validation_expr: String, + /// Data Type + pub data_type: DataType, +} + +impl GeneratedColumn { + /// Create a new invariant + pub fn new(field_name: &str, sql_generation: &str, data_type: &DataType) -> Self { + Self { + name: field_name.to_string(), + generation_expr: sql_generation.to_string(), + validation_expr: format!("{field_name} <=> {sql_generation}"), // update to + data_type: data_type.clone(), + } + } + + pub fn get_generation_expression(&self) -> &str { + &self.generation_expr + } +} + +impl DataCheck for GeneratedColumn { + fn get_name(&self) -> &str { + &self.name + } + + fn get_expression(&self) -> &str { + &self.validation_expr + } + + fn as_any(&self) -> &dyn Any { + self + } +} diff --git a/crates/core/src/table/mod.rs b/crates/core/src/table/mod.rs index cbeccf893b..9a5ac478c0 100644 --- a/crates/core/src/table/mod.rs +++ b/crates/core/src/table/mod.rs @@ -1,6 +1,5 @@ //! Delta Table read and write implementation -use std::any::Any; use std::cmp::{min, Ordering}; use std::collections::HashMap; use std::fmt; @@ -18,9 +17,11 @@ use self::state::DeltaTableState; use crate::kernel::{ CommitInfo, DataCheck, DataType, LogicalFile, Metadata, Protocol, StructType, Transaction, }; -use crate::logstore::{extract_version_from_filename, LogStoreConfig, LogStoreRef}; +use crate::logstore::{ + commit_uri_from_version, extract_version_from_filename, LogStoreConfig, LogStoreExt, + LogStoreRef, ObjectStoreRef, +}; use crate::partitions::PartitionFilter; -use crate::storage::{commit_uri_from_version, ObjectStoreRef}; use crate::{DeltaResult, DeltaTableError}; // NOTE: this use can go away when peek_next_commit is removed off of [DeltaTable] @@ -31,180 +32,10 @@ pub mod config; pub mod state; pub mod state_arrow; -/// Metadata for a checkpoint file -#[derive(Serialize, Deserialize, Debug, Default, Clone, Copy)] -#[serde(rename_all = "camelCase")] -pub struct CheckPoint { - /// Delta table version - pub(crate) version: i64, // 20 digits decimals - /// The number of actions that are stored in the checkpoint. - pub(crate) size: i64, - #[serde(skip_serializing_if = "Option::is_none")] - /// The number of fragments if the last checkpoint was written in multiple parts. This field is optional. - pub(crate) parts: Option, // 10 digits decimals - #[serde(skip_serializing_if = "Option::is_none")] - /// The number of bytes of the checkpoint. This field is optional. - pub(crate) size_in_bytes: Option, - #[serde(skip_serializing_if = "Option::is_none")] - /// The number of AddFile actions in the checkpoint. This field is optional. - pub(crate) num_of_add_files: Option, -} - -#[derive(Default)] -/// Builder for CheckPoint -pub struct CheckPointBuilder { - /// Delta table version - pub(crate) version: i64, // 20 digits decimals - /// The number of actions that are stored in the checkpoint. - pub(crate) size: i64, - /// The number of fragments if the last checkpoint was written in multiple parts. This field is optional. - pub(crate) parts: Option, // 10 digits decimals - /// The number of bytes of the checkpoint. This field is optional. - pub(crate) size_in_bytes: Option, - /// The number of AddFile actions in the checkpoint. This field is optional. - pub(crate) num_of_add_files: Option, -} - -impl CheckPointBuilder { - /// Creates a new [`CheckPointBuilder`] instance with the provided `version` and `size`. - /// Size is the total number of actions in the checkpoint. See size_in_bytes for total size in bytes. - pub fn new(version: i64, size: i64) -> Self { - CheckPointBuilder { - version, - size, - parts: None, - size_in_bytes: None, - num_of_add_files: None, - } - } - - /// The number of fragments if the last checkpoint was written in multiple parts. This field is optional. - pub fn with_parts(mut self, parts: u32) -> Self { - self.parts = Some(parts); - self - } - - /// The number of bytes of the checkpoint. This field is optional. - pub fn with_size_in_bytes(mut self, size_in_bytes: i64) -> Self { - self.size_in_bytes = Some(size_in_bytes); - self - } - - /// The number of AddFile actions in the checkpoint. This field is optional. - pub fn with_num_of_add_files(mut self, num_of_add_files: i64) -> Self { - self.num_of_add_files = Some(num_of_add_files); - self - } - - /// Build the final [`CheckPoint`] struct. - pub fn build(self) -> CheckPoint { - CheckPoint { - version: self.version, - size: self.size, - parts: self.parts, - size_in_bytes: self.size_in_bytes, - num_of_add_files: self.num_of_add_files, - } - } -} - -impl CheckPoint { - /// Creates a new checkpoint from the given parameters. - pub fn new(version: i64, size: i64, parts: Option) -> Self { - Self { - version, - size, - parts: parts.or(None), - size_in_bytes: None, - num_of_add_files: None, - } - } -} - -impl PartialEq for CheckPoint { - fn eq(&self, other: &Self) -> bool { - self.version == other.version - } -} - -impl Eq for CheckPoint {} - -/// A constraint in a check constraint -#[derive(Eq, PartialEq, Debug, Default, Clone)] -pub struct Constraint { - /// The full path to the field. - pub name: String, - /// The SQL string that must always evaluate to true. - pub expr: String, -} - -impl Constraint { - /// Create a new invariant - pub fn new(field_name: &str, invariant_sql: &str) -> Self { - Self { - name: field_name.to_string(), - expr: invariant_sql.to_string(), - } - } -} - -impl DataCheck for Constraint { - fn get_name(&self) -> &str { - &self.name - } - - fn get_expression(&self) -> &str { - &self.expr - } - - fn as_any(&self) -> &dyn Any { - self - } -} - -/// A generated column -#[derive(Eq, PartialEq, Debug, Clone)] -pub struct GeneratedColumn { - /// The full path to the field. - pub name: String, - /// The SQL string that generate the column value. - pub generation_expr: String, - /// The SQL string that must always evaluate to true. - pub validation_expr: String, - /// Data Type - pub data_type: DataType, -} +mod columns; -impl GeneratedColumn { - /// Create a new invariant - pub fn new(field_name: &str, sql_generation: &str, data_type: &DataType) -> Self { - Self { - name: field_name.to_string(), - generation_expr: sql_generation.to_string(), - validation_expr: format!("{field_name} = {sql_generation} OR ({field_name} IS NULL AND {sql_generation} IS NULL)"), - // validation_expr: format!("{field_name} <=> {sql_generation}"), // update to - data_type: data_type.clone() - } - } - - pub fn get_generation_expression(&self) -> &str { - &self.generation_expr - } -} - -impl DataCheck for GeneratedColumn { - fn get_name(&self) -> &str { - &self.name - } - - fn get_expression(&self) -> &str { - &self.validation_expr - } - - fn as_any(&self) -> &dyn Any { - self - } -} +// Re-exposing for backwards compatibility +pub use columns::*; /// Return partition fields along with their data type from the current schema. pub(crate) fn get_partition_col_data_types<'a>( @@ -230,6 +61,11 @@ pub(crate) fn get_partition_col_data_types<'a>( } /// In memory representation of a Delta Table +/// +/// A DeltaTable is a purely logical concept that represents a dataset that can ewvolve over time. +/// To attain concrete information about a table a snapshot need to be loaded. +/// Most commonly this is the latest state of the tablem but may also loaded for a specific +/// version or point in time. #[derive(Clone)] pub struct DeltaTable { /// The state of the table as of the most recent loaded Delta log entry. @@ -280,12 +116,9 @@ impl<'de> Deserialize<'de> for DeltaTable { let storage_config: LogStoreConfig = seq .next_element()? .ok_or_else(|| A::Error::invalid_length(0, &self))?; - let log_store = crate::logstore::logstore_for( - storage_config.location, - storage_config.options, - None, - ) - .map_err(|_| A::Error::custom("Failed deserializing LogStore"))?; + let log_store = + crate::logstore::logstore_for(storage_config.location, storage_config.options) + .map_err(|_| A::Error::custom("Failed deserializing LogStore"))?; let table = DeltaTable { state, @@ -348,17 +181,17 @@ impl DeltaTable { /// returns the latest available version of the table pub async fn get_latest_version(&self) -> Result { - self.log_store.get_latest_version(self.version()).await - } - - /// returns the earliest available version of the table - pub async fn get_earliest_version(&self) -> Result { - self.log_store.get_earliest_version(self.version()).await + self.log_store + .get_latest_version(self.version().unwrap_or(0)) + .await } - /// Currently loaded version of the table - pub fn version(&self) -> i64 { - self.state.as_ref().map(|s| s.version()).unwrap_or(-1) + /// Currently loaded version of the table - if any. + /// + /// This will return the latest version of the table if it has been loaded. + /// Returns `None` if the table has not been loaded. + pub fn version(&self) -> Option { + self.state.as_ref().map(|s| s.version()) } /// Load DeltaTable with data from latest checkpoint @@ -372,18 +205,6 @@ impl DeltaTable { self.update_incremental(None).await } - #[deprecated( - since = "0.22.4", - note = "peek_next_commit has moved to the logstore, use table.log_store().peek_next_commit() instead please :)" - )] - /// Get the list of actions for the next commit - pub async fn peek_next_commit( - &self, - current_version: i64, - ) -> Result { - self.log_store().peek_next_commit(current_version).await - } - /// Updates the DeltaTable to the latest version by incrementally applying newer versions. /// It assumes that the table is already updated to the current version `self.version`. pub async fn update_incremental( @@ -391,15 +212,11 @@ impl DeltaTable { max_version: Option, ) -> Result<(), DeltaTableError> { match self.state.as_mut() { - Some(state) => state.update(self.log_store.clone(), max_version).await, + Some(state) => state.update(&self.log_store, max_version).await, _ => { - let state = DeltaTableState::try_new( - &Path::default(), - self.log_store.object_store(None), - self.config.clone(), - max_version, - ) - .await?; + let state = + DeltaTableState::try_new(&self.log_store, self.config.clone(), max_version) + .await?; self.state = Some(state); Ok(()) } @@ -443,7 +260,7 @@ impl DeltaTable { .snapshot()? .snapshot .snapshot() - .commit_infos(self.object_store(), limit) + .commit_infos(&self.log_store(), limit) .await? .try_collect::>() .await?; @@ -531,12 +348,15 @@ impl DeltaTable { Ok(self.snapshot()?.metadata()) } + #[deprecated( + since = "0.27.0", + note = "Use `snapshot()?.transaction_version(app_id)` instead." + )] /// Returns the current version of the DeltaTable based on the loaded metadata. pub fn get_app_transaction_version(&self) -> HashMap { self.state .as_ref() - .and_then(|s| s.app_transaction_version().ok()) - .map(|it| it.map(|t| (t.app_id.clone(), t)).collect()) + .and_then(|s| s.snapshot.transactions.clone()) .unwrap_or_default() } @@ -562,13 +382,24 @@ impl DeltaTable { ) -> Result<(), DeltaTableError> { let mut min_version: i64 = -1; let log_store = self.log_store(); - let prefix = Some(log_store.log_path()); + let prefix = log_store.log_path(); let offset_path = commit_uri_from_version(min_version); let object_store = log_store.object_store(None); - let mut files = object_store.list_with_offset(prefix, &offset_path); + let mut files = object_store.list_with_offset(Some(prefix), &offset_path); while let Some(obj_meta) = files.next().await { let obj_meta = obj_meta?; + let location_path: Path = obj_meta.location.clone(); + let part_count = location_path.prefix_match(prefix).unwrap().count(); + if part_count > 1 { + // Per the spec, ignore any files in subdirectories. + // Spark may create these as uncommitted transactions which we don't want + // + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#delta-log-entries + // "Delta files are stored as JSON in a directory at the *root* of the table + // named _delta_log, and ... make up the log of all changes that have occurred to a table." + continue; + } if let Some(log_version) = extract_version_from_filename(obj_meta.location.as_ref()) { if min_version == -1 { min_version = log_version @@ -580,7 +411,15 @@ impl DeltaTable { break; } } - let mut max_version = self.get_latest_version().await?; + let mut max_version = match self.get_latest_version().await { + Ok(version) => version, + Err(DeltaTableError::InvalidVersion(_)) => { + return Err(DeltaTableError::NotATable( + log_store.table_root_url().to_string(), + )) + } + Err(e) => return Err(e), + }; let mut version = min_version; let lowest_table_version = min_version; let target_ts = datetime.timestamp_millis(); @@ -615,7 +454,7 @@ impl DeltaTable { impl fmt::Display for DeltaTable { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { writeln!(f, "DeltaTable({})", self.table_uri())?; - writeln!(f, "\tversion: {}", self.version()) + writeln!(f, "\tversion: {:?}", self.version()) } } @@ -668,21 +507,4 @@ mod tests { .unwrap(); (dt, tmp_dir) } - - #[test] - fn checkpoint_should_serialize_in_camel_case() { - let checkpoint = CheckPoint { - version: 1, - size: 1, - parts: None, - size_in_bytes: Some(1), - num_of_add_files: Some(1), - }; - - let checkpoint_json_serialized = - serde_json::to_string(&checkpoint).expect("could not serialize to json"); - - assert!(checkpoint_json_serialized.contains("sizeInBytes")); - assert!(checkpoint_json_serialized.contains("numOfAddFiles")); - } } diff --git a/crates/core/src/table/state.rs b/crates/core/src/table/state.rs index 2a25399c42..94824cf726 100644 --- a/crates/core/src/table/state.rs +++ b/crates/core/src/table/state.rs @@ -1,11 +1,10 @@ //! The module for delta table state. use std::collections::{HashMap, HashSet}; -use std::sync::Arc; use chrono::Utc; use futures::TryStreamExt; -use object_store::{path::Path, ObjectStore}; +use object_store::path::Path; use serde::{Deserialize, Serialize}; use super::{config::TableConfig, get_partition_col_data_types, DeltaTableConfig}; @@ -13,7 +12,7 @@ use super::{config::TableConfig, get_partition_col_data_types, DeltaTableConfig} use crate::kernel::Action; use crate::kernel::{ ActionType, Add, AddCDCFile, DataType, EagerSnapshot, LogDataHandler, LogicalFile, Metadata, - Protocol, Remove, StructType, Transaction, + Protocol, Remove, StructType, }; use crate::logstore::LogStore; use crate::partitions::{DeltaTablePartition, PartitionFilter}; @@ -29,14 +28,12 @@ pub struct DeltaTableState { impl DeltaTableState { /// Create a new DeltaTableState pub async fn try_new( - table_root: &Path, - store: Arc, + log_store: &dyn LogStore, config: DeltaTableConfig, version: Option, ) -> DeltaResult { let snapshot = EagerSnapshot::try_new_with_visitor( - table_root, - store.clone(), + log_store, config, version, HashSet::from([ActionType::Txn]), @@ -59,8 +56,8 @@ impl DeltaTableState { /// Construct a delta table state object from a list of actions #[cfg(test)] - pub fn from_actions(actions: Vec) -> DeltaResult { - use crate::operations::transaction::CommitData; + pub fn from_actions(actions: Vec, table_root: &Path) -> DeltaResult { + use crate::kernel::transaction::CommitData; use crate::protocol::{DeltaOperation, SaveMode}; let metadata = actions @@ -90,7 +87,7 @@ impl DeltaTableState { Vec::new(), )]; - let snapshot = EagerSnapshot::new_test(&commit_data).unwrap(); + let snapshot = EagerSnapshot::new_test(&commit_data, table_root).unwrap(); Ok(Self { snapshot }) } @@ -102,12 +99,12 @@ impl DeltaTableState { /// Full list of tombstones (remove actions) representing files removed from table state). pub async fn all_tombstones( &self, - store: Arc, + log_store: &dyn LogStore, ) -> DeltaResult> { Ok(self .snapshot .snapshot() - .tombstones(store)? + .tombstones(log_store)? .try_collect::>() .await? .into_iter() @@ -118,14 +115,14 @@ impl DeltaTableState { /// The retention period is set by `deletedFileRetentionDuration` with default value of 1 week. pub async fn unexpired_tombstones( &self, - store: Arc, + log_store: &dyn LogStore, ) -> DeltaResult> { let retention_timestamp = Utc::now().timestamp_millis() - self .table_config() .deleted_file_retention_duration() .as_millis() as i64; - let tombstones = self.all_tombstones(store).await?.collect::>(); + let tombstones = self.all_tombstones(log_store).await?.collect::>(); Ok(tombstones .into_iter() .filter(move |t| t.deletion_timestamp.unwrap_or(0) > retention_timestamp)) @@ -161,9 +158,15 @@ impl DeltaTableState { .map(|add| add.object_store_path()) } - /// HashMap containing the last transaction stored for every application. - pub fn app_transaction_version(&self) -> DeltaResult + '_> { - self.snapshot.transactions() + /// Get the transaction version for the given application ID. + /// + /// Returns `None` if the application ID is not found. + pub async fn transaction_version( + &self, + _log_store: &dyn LogStore, + app_id: impl AsRef, + ) -> DeltaResult> { + self.snapshot.transaction_version(app_id).await } /// The most recent protocol of the table. @@ -199,7 +202,7 @@ impl DeltaTableState { /// Update the state of the table to the given version. pub async fn update( &mut self, - log_store: Arc, + log_store: &dyn LogStore, version: Option, ) -> Result<(), DeltaTableError> { self.snapshot.update(log_store, version).await?; diff --git a/crates/core/src/table/state_arrow.rs b/crates/core/src/table/state_arrow.rs index c6eb72df37..a4e135a654 100644 --- a/crates/core/src/table/state_arrow.rs +++ b/crates/core/src/table/state_arrow.rs @@ -14,6 +14,7 @@ use arrow_array::{ use arrow_cast::cast; use arrow_cast::parse::Parser; use arrow_schema::{DataType, Field, Fields, TimeUnit}; +use delta_kernel::engine::arrow_conversion::TryIntoArrow as _; use delta_kernel::table_features::{validate_schema_column_mapping, ColumnMappingMode}; use itertools::Itertools; @@ -155,7 +156,7 @@ impl DeltaTableState { .ok_or(DeltaTableError::MetadataError(format!( "Invalid partition column {name}" )))?; - Ok(field.data_type().try_into()?) + Ok(field.data_type().try_into_arrow()?) }, ) .collect::>()?; @@ -463,7 +464,7 @@ impl DeltaTableState { Ok(Arc::new(arrow::array::Int64Array::from(values))) })?); - let arrow_type: arrow::datatypes::DataType = datatype.try_into()?; + let arrow_type: arrow::datatypes::DataType = datatype.try_into_arrow()?; // Min and max are collected for primitive values, not list or maps let min_values = if matches!(datatype, DeltaDataType::Primitive(_)) { diff --git a/crates/core/src/test_utils/factories/actions.rs b/crates/core/src/test_utils/factories/actions.rs index 23f7206d89..50c0aabb16 100644 --- a/crates/core/src/test_utils/factories/actions.rs +++ b/crates/core/src/test_utils/factories/actions.rs @@ -9,9 +9,9 @@ use object_store::ObjectMeta; use super::{get_parquet_bytes, DataFactory, FileStats}; use crate::kernel::arrow::extract::{self as ex}; use crate::kernel::partitions_schema; +use crate::kernel::transaction::PROTOCOL; use crate::kernel::{Add, Metadata, Protocol, Remove, StructType}; -use crate::operations::transaction::PROTOCOL; -use delta_kernel::table_features::{ReaderFeatures, WriterFeatures}; +use delta_kernel::table_features::{ReaderFeature, WriterFeature}; pub struct ActionFactory; @@ -34,7 +34,6 @@ impl ActionFactory { deletion_vector: None, base_row_id: None, clustering_provider: None, - stats_parsed: None, } } @@ -86,7 +85,7 @@ impl ActionFactory { let data = get_parquet_bytes(&batch).unwrap(); let meta = ObjectMeta { location: path.clone(), - size: data.len(), + size: data.len() as u64, last_modified: Utc::now(), e_tag: None, version: None, @@ -101,8 +100,8 @@ impl ActionFactory { pub fn protocol( max_reader: Option, max_writer: Option, - reader_features: Option>, - writer_features: Option>, + reader_features: Option>, + writer_features: Option>, ) -> Protocol { Protocol { min_reader_version: max_reader.unwrap_or(PROTOCOL.default_reader_version()), diff --git a/crates/core/src/test_utils/factories/data.rs b/crates/core/src/test_utils/factories/data.rs index d69869ae92..f96a5aa18d 100644 --- a/crates/core/src/test_utils/factories/data.rs +++ b/crates/core/src/test_utils/factories/data.rs @@ -5,6 +5,7 @@ use arrow_arith::aggregate::{max as arrow_max, max_string, min as arrow_min, min use arrow_array::*; use arrow_schema::DataType as ArrowDataType; use bytes::Bytes; +use delta_kernel::engine::arrow_conversion::TryIntoArrow as _; use delta_kernel::expressions::Scalar; use parquet::arrow::arrow_writer::ArrowWriter; use parquet::file::properties::WriterProperties; @@ -64,7 +65,9 @@ fn generate_random_batch( ) }) .collect::>>() - .map(|columns| RecordBatch::try_new(Arc::new(schema.try_into().unwrap()), columns).unwrap()) + .map(|columns| { + RecordBatch::try_new(Arc::new(schema.try_into_arrow().unwrap()), columns).unwrap() + }) } pub fn generate_random_array( diff --git a/crates/core/src/test_utils/mod.rs b/crates/core/src/test_utils/mod.rs index 0d3ff9ed65..24b4cb908b 100644 --- a/crates/core/src/test_utils/mod.rs +++ b/crates/core/src/test_utils/mod.rs @@ -1,5 +1,170 @@ mod factories; -pub use factories::*; +use std::{collections::HashMap, path::PathBuf, process::Command}; + +use url::Url; + +pub use self::factories::*; +use crate::DeltaTableBuilder; pub type TestResult = Result>; + +/// Reference tables from the test data folder +pub enum TestTables { + Simple, + SimpleWithCheckpoint, + SimpleCommit, + Golden, + Delta0_8_0Partitioned, + Delta0_8_0SpecialPartitioned, + Checkpoints, + LatestNotCheckpointed, + WithDvSmall, + Custom(String), +} + +impl TestTables { + pub fn as_path(&self) -> PathBuf { + let data_path = find_git_root().join("crates/test/tests/data"); + match self { + Self::Simple => data_path.join("simple_table"), + Self::SimpleWithCheckpoint => data_path.join("simple_table_with_checkpoint"), + Self::SimpleCommit => data_path.join("simple_commit"), + Self::Golden => data_path.join("golden/data-reader-array-primitives"), + Self::Delta0_8_0Partitioned => data_path.join("delta-0.8.0-partitioned"), + Self::Delta0_8_0SpecialPartitioned => data_path.join("delta-0.8.0-special-partition"), + Self::Checkpoints => data_path.join("checkpoints"), + Self::LatestNotCheckpointed => data_path.join("latest_not_checkpointed"), + Self::WithDvSmall => data_path.join("table-with-dv-small"), + // the data path for upload does not apply to custom tables. + Self::Custom(_) => todo!(), + } + } + + pub fn as_name(&self) -> String { + match self { + Self::Simple => "simple".into(), + Self::SimpleWithCheckpoint => "simple_table_with_checkpoint".into(), + Self::SimpleCommit => "simple_commit".into(), + Self::Golden => "golden".into(), + Self::Delta0_8_0Partitioned => "delta-0.8.0-partitioned".into(), + Self::Delta0_8_0SpecialPartitioned => "delta-0.8.0-special-partition".into(), + Self::Checkpoints => "checkpoints".into(), + Self::LatestNotCheckpointed => "latest_not_checkpointed".into(), + Self::WithDvSmall => "table-with-dv-small".into(), + Self::Custom(name) => name.to_owned(), + } + } + + pub fn uri_for_table(&self, root_uri: impl AsRef) -> String { + let root_uri = root_uri.as_ref(); + if root_uri.ends_with('/') { + format!("{}{}", root_uri, self.as_name()) + } else { + format!("{}/{}", root_uri, self.as_name()) + } + } + + pub fn table_builder(&self) -> DeltaTableBuilder { + let url = Url::from_directory_path(self.as_path()).unwrap(); + DeltaTableBuilder::from_uri(url).with_allow_http(true) + } +} + +fn find_git_root() -> PathBuf { + let output = Command::new("git") + .arg("rev-parse") + .arg("--show-toplevel") + .output() + .unwrap(); + PathBuf::from(String::from_utf8(output.stdout).unwrap().trim()) +} + +/// Test fixture that allows setting environment variables for the duration of a test. +/// +/// Existing environment variables that are overwritten will be restored when the fixture is dropped. +pub fn with_env(vars: Vec<(&str, &str)>) -> impl Drop { + // Store the original values before modifying + let original_values: HashMap> = vars + .iter() + .map(|(key, _)| (key.to_string(), std::env::var(key).ok())) + .collect(); + + // Set all the new environment variables + for (key, value) in vars { + std::env::set_var(key, value); + } + + // Create a cleanup struct that will restore original values when dropped + struct EnvCleanup(HashMap>); + + impl Drop for EnvCleanup { + fn drop(&mut self) { + for (key, maybe_value) in self.0.iter() { + match maybe_value { + Some(value) => std::env::set_var(key, value), + None => std::env::remove_var(key), + } + } + } + } + + EnvCleanup(original_values) +} + +#[macro_export] +macro_rules! assert_batches_sorted_eq { + ($EXPECTED_LINES: expr, $CHUNKS: expr) => { + let mut expected_lines: Vec = $EXPECTED_LINES.iter().map(|&s| s.into()).collect(); + + // sort except for header + footer + let num_lines = expected_lines.len(); + if num_lines > 3 { + expected_lines.as_mut_slice()[2..num_lines - 1].sort_unstable() + } + + let formatted = arrow::util::pretty::pretty_format_batches($CHUNKS) + .unwrap() + .to_string(); + // fix for windows: \r\n --> + + let mut actual_lines: Vec<&str> = formatted.trim().lines().collect(); + + // sort except for header + footer + let num_lines = actual_lines.len(); + if num_lines > 3 { + actual_lines.as_mut_slice()[2..num_lines - 1].sort_unstable() + } + + assert_eq!( + expected_lines, actual_lines, + "\n\nexpected:\n\n{expected_lines:#?}\nactual:\n\n{actual_lines:#?}\n\n", + ); + }; +} + +pub use assert_batches_sorted_eq; + +#[cfg(test)] +mod tests { + use std::env; + + use super::*; + + #[test] + fn test_api_with_env() { + let _env = with_env(vec![ + ("API_KEY", "test_key"), + ("API_URL", "http://test.example.com"), + ]); + + // Test code using these environment variables + assert_eq!(env::var("API_KEY").unwrap(), "test_key"); + assert_eq!(env::var("API_URL").unwrap(), "http://test.example.com"); + + drop(_env); + + assert!(env::var("API_KEY").is_err()); + assert!(env::var("API_URL").is_err()); + } +} diff --git a/crates/core/src/writer/json.rs b/crates/core/src/writer/json.rs index a4837eeb6d..e2261ecbff 100644 --- a/crates/core/src/writer/json.rs +++ b/crates/core/src/writer/json.rs @@ -1,11 +1,11 @@ //! Main writer API to write json messages to delta table use std::collections::HashMap; -use std::convert::TryFrom; use std::sync::Arc; use arrow::datatypes::{Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; use arrow::record_batch::*; use bytes::Bytes; +use delta_kernel::engine::arrow_conversion::TryIntoArrow as _; use delta_kernel::expressions::Scalar; use indexmap::IndexMap; use object_store::path::Path; @@ -24,8 +24,8 @@ use super::utils::{ }; use super::{DeltaWriter, DeltaWriterError, WriteMode}; use crate::errors::DeltaTableError; -use crate::kernel::{scalars::ScalarExt, Add, PartitionsExt, StructType}; -use crate::storage::retry_ext::ObjectStoreRetryExt; +use crate::kernel::{scalars::ScalarExt, Add, PartitionsExt}; +use crate::logstore::ObjectStoreRetryExt; use crate::table::builder::DeltaTableBuilder; use crate::writer::utils::ShareableBuffer; use crate::DeltaTable; @@ -258,9 +258,11 @@ impl JsonWriter { .table .schema() .expect("Failed to unwrap schema for table"); - >::try_from(schema) - .expect("Failed to coerce delta schema to arrow") - .into() + Arc::new( + schema + .try_into_arrow() + .expect("Failed to coerce delta schema to arrow"), + ) } fn divide_by_partition_values( @@ -478,7 +480,7 @@ mod tests { .await .unwrap(); table.load().await.expect("Failed to load table"); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); table } @@ -487,7 +489,7 @@ mod tests { let table_dir = tempfile::tempdir().unwrap(); let table = get_test_table(&table_dir).await; let schema = table.schema().unwrap(); - let arrow_schema = >::try_from(schema).unwrap(); + let arrow_schema: ArrowSchema = schema.try_into_arrow().unwrap(); let mut writer = JsonWriter::try_new( table.table_uri(), Arc::new(arrow_schema), @@ -563,8 +565,7 @@ mod tests { let table_dir = tempfile::tempdir().unwrap(); let table = get_test_table(&table_dir).await; - let arrow_schema = - >::try_from(table.schema().unwrap()).unwrap(); + let arrow_schema: ArrowSchema = table.schema().unwrap().try_into_arrow().unwrap(); let mut writer = JsonWriter::try_new( table.table_uri(), Arc::new(arrow_schema), @@ -601,8 +602,7 @@ mod tests { let table_dir = tempfile::tempdir().unwrap(); let table = get_test_table(&table_dir).await; - let arrow_schema = - >::try_from(table.schema().unwrap()).unwrap(); + let arrow_schema: ArrowSchema = table.schema().unwrap().try_into_arrow().unwrap(); let mut writer = JsonWriter::try_new( table.table_uri(), Arc::new(arrow_schema), @@ -636,13 +636,14 @@ mod tests { } } + #[cfg(feature = "datafusion")] #[tokio::test] async fn test_json_write_mismatched_schema() { let table_dir = tempfile::tempdir().unwrap(); let mut table = get_test_table(&table_dir).await; let schema = table.schema().unwrap(); - let arrow_schema = >::try_from(schema).unwrap(); + let arrow_schema: ArrowSchema = schema.try_into_arrow().unwrap(); let mut writer = JsonWriter::try_new( table.table_uri(), Arc::new(arrow_schema), @@ -674,10 +675,11 @@ mod tests { // TODO This should fail because we haven't asked to evolve the schema writer.write(vec![second_data]).await.unwrap(); writer.flush_and_commit(&mut table).await.unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); } } + #[cfg(feature = "datafusion")] #[tokio::test] async fn test_json_write_checkpoint() { use std::fs; @@ -702,7 +704,7 @@ mod tests { .with_configuration(config) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let mut writer = JsonWriter::for_table(&table).unwrap(); let data = serde_json::json!( { @@ -726,6 +728,7 @@ mod tests { assert_eq!(entries.len(), 1); } + #[cfg(feature = "datafusion")] #[tokio::test] async fn test_json_write_data_skipping_stats_columns() { let table_dir = tempfile::tempdir().unwrap(); @@ -745,7 +748,7 @@ mod tests { .with_configuration(config) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let mut writer = JsonWriter::for_table(&table).unwrap(); let data = serde_json::json!( { @@ -757,7 +760,7 @@ mod tests { writer.write(vec![data]).await.unwrap(); writer.flush_and_commit(&mut table).await.unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let add_actions = table.state.unwrap().file_actions().unwrap(); assert_eq!(add_actions.len(), 1); let expected_stats = "{\"numRecords\":1,\"minValues\":{\"id\":\"A\",\"value\":42},\"maxValues\":{\"id\":\"A\",\"value\":42},\"nullCount\":{\"id\":0,\"value\":0}}"; @@ -774,6 +777,7 @@ mod tests { ); } + #[cfg(feature = "datafusion")] #[tokio::test] async fn test_json_write_data_skipping_num_indexed_cols() { let table_dir = tempfile::tempdir().unwrap(); @@ -793,7 +797,7 @@ mod tests { .with_configuration(config) .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let mut writer = JsonWriter::for_table(&table).unwrap(); let data = serde_json::json!( { @@ -805,7 +809,7 @@ mod tests { writer.write(vec![data]).await.unwrap(); writer.flush_and_commit(&mut table).await.unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let add_actions = table.state.unwrap().file_actions().unwrap(); assert_eq!(add_actions.len(), 1); let expected_stats = "{\"numRecords\":1,\"minValues\":{\"id\":\"A\"},\"maxValues\":{\"id\":\"A\"},\"nullCount\":{\"id\":0}}"; diff --git a/crates/core/src/writer/mod.rs b/crates/core/src/writer/mod.rs index cd87459c2f..169a72bdf8 100644 --- a/crates/core/src/writer/mod.rs +++ b/crates/core/src/writer/mod.rs @@ -7,8 +7,8 @@ use parquet::errors::ParquetError; use serde_json::Value; use crate::errors::DeltaTableError; +use crate::kernel::transaction::{CommitBuilder, CommitProperties}; use crate::kernel::{Action, Add}; -use crate::operations::transaction::{CommitBuilder, CommitProperties}; use crate::protocol::{ColumnCountStat, DeltaOperation, SaveMode}; use crate::DeltaTable; diff --git a/crates/core/src/writer/record_batch.rs b/crates/core/src/writer/record_batch.rs index 20cb544290..f33d25cf90 100644 --- a/crates/core/src/writer/record_batch.rs +++ b/crates/core/src/writer/record_batch.rs @@ -13,6 +13,7 @@ use arrow_row::{RowConverter, SortField}; use arrow_schema::{ArrowError, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef}; use arrow_select::take::take; use bytes::Bytes; +use delta_kernel::engine::arrow_conversion::{TryIntoArrow, TryIntoKernel}; use delta_kernel::expressions::Scalar; use indexmap::IndexMap; use object_store::{path::Path, ObjectStore}; @@ -28,9 +29,9 @@ use super::utils::{ }; use super::{DeltaWriter, DeltaWriterError, WriteMode}; use crate::errors::DeltaTableError; -use crate::kernel::{scalars::ScalarExt, Action, Add, PartitionsExt, StructType}; +use crate::kernel::{scalars::ScalarExt, Action, Add, PartitionsExt}; +use crate::logstore::ObjectStoreRetryExt; use crate::operations::cast::merge_schema::merge_arrow_schema; -use crate::storage::ObjectStoreRetryExt; use crate::table::builder::DeltaTableBuilder; use crate::table::config::DEFAULT_NUM_INDEX_COLS; use crate::DeltaTable; @@ -102,8 +103,7 @@ impl RecordBatchWriter { pub fn for_table(table: &DeltaTable) -> Result { // Initialize an arrow schema ref from the delta table schema let metadata = table.metadata()?; - let arrow_schema = - >::try_from(&metadata.schema()?.clone())?; + let arrow_schema: ArrowSchema = (&metadata.schema()?).try_into_arrow()?; let arrow_schema_ref = Arc::new(arrow_schema); let partition_columns = metadata.partition_columns.clone(); @@ -275,7 +275,7 @@ impl DeltaWriter for RecordBatchWriter { let mut adds: Vec = self.flush().await?.drain(..).map(Action::Add).collect(); if self.arrow_schema_ref != self.original_schema_ref && self.should_evolve { - let schema: StructType = self.arrow_schema_ref.clone().try_into()?; + let schema: StructType = self.arrow_schema_ref.clone().try_into_kernel()?; if !self.partition_columns.is_empty() { return Err(DeltaTableError::Generic( "Merging Schemas with partition columns present is currently unsupported" @@ -499,6 +499,7 @@ mod tests { use arrow::json::ReaderBuilder; use arrow_array::{Int32Array, RecordBatch, StringArray}; use arrow_schema::{DataType, Field, Schema as ArrowSchema}; + use delta_kernel::schema::StructType; use std::path::Path; #[tokio::test] @@ -592,7 +593,7 @@ mod tests { "metadata" : {"some-key" : "some-value"}}"# .as_bytes(); - let schema: ArrowSchema = (&delta_schema).try_into().unwrap(); + let schema: ArrowSchema = (&delta_schema).try_into_arrow().unwrap(); // Using a batch size of two since the buf above only has two records let mut decoder = ReaderBuilder::new(Arc::new(schema)) @@ -722,6 +723,7 @@ mod tests { // The following sets of tests are related to #1386 and mergeSchema support // + #[cfg(feature = "datafusion")] mod schema_evolution { use itertools::Itertools; @@ -790,7 +792,7 @@ mod tests { .await .unwrap(); table.load().await.expect("Failed to load table"); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let batch = get_record_batch(None, false); let mut writer = RecordBatchWriter::for_table(&table).unwrap(); @@ -799,7 +801,7 @@ mod tests { let version = writer.flush_and_commit(&mut table).await.unwrap(); assert_eq!(version, 1); table.load().await.expect("Failed to load table"); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); // Create a second batch with a different schema let second_schema = Arc::new(ArrowSchema::new(vec![ @@ -825,7 +827,7 @@ mod tests { let version = writer.flush_and_commit(&mut table).await.unwrap(); assert_eq!(version, 2); table.load().await.expect("Failed to load table"); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); let new_schema = table.metadata().unwrap().schema().unwrap(); let expected_columns = vec!["id", "value", "modified", "vid", "name"]; @@ -851,7 +853,7 @@ mod tests { .await .unwrap(); table.load().await.expect("Failed to load table"); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let batch = get_record_batch(None, false); let mut writer = RecordBatchWriter::for_table(&table).unwrap(); @@ -860,7 +862,7 @@ mod tests { let version = writer.flush_and_commit(&mut table).await.unwrap(); assert_eq!(version, 1); table.load().await.expect("Failed to load table"); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); // Create a second batch with appended columns let second_batch = { @@ -983,7 +985,7 @@ mod tests { .await .unwrap(); table.load().await.expect("Failed to load table"); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); // Hand-crafting the first RecordBatch to ensure that a write with non-nullable columns // works properly before attempting the second write @@ -1032,6 +1034,7 @@ mod tests { } } + #[cfg(feature = "datafusion")] #[tokio::test] async fn test_write_data_skipping_stats_columns() { let batch = get_record_batch(None, false); @@ -1063,7 +1066,7 @@ mod tests { assert_eq!(partitions[0].record_batch, batch); writer.write(batch).await.unwrap(); writer.flush_and_commit(&mut table).await.unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let add_actions = table.state.unwrap().file_actions().unwrap(); assert_eq!(add_actions.len(), 1); let expected_stats ="{\"numRecords\":11,\"minValues\":{\"value\":1,\"id\":\"A\"},\"maxValues\":{\"id\":\"B\",\"value\":11},\"nullCount\":{\"id\":0,\"value\":0}}"; @@ -1080,6 +1083,7 @@ mod tests { ); } + #[cfg(feature = "datafusion")] #[tokio::test] async fn test_write_data_skipping_num_indexed_colsn() { let batch = get_record_batch(None, false); @@ -1111,7 +1115,7 @@ mod tests { assert_eq!(partitions[0].record_batch, batch); writer.write(batch).await.unwrap(); writer.flush_and_commit(&mut table).await.unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let add_actions = table.state.unwrap().file_actions().unwrap(); assert_eq!(add_actions.len(), 1); let expected_stats = "{\"numRecords\":11,\"minValues\":{\"id\":\"A\"},\"maxValues\":{\"id\":\"B\"},\"nullCount\":{\"id\":0}}"; diff --git a/crates/core/src/writer/stats.rs b/crates/core/src/writer/stats.rs index 7859f27121..aa2b6a3134 100644 --- a/crates/core/src/writer/stats.rs +++ b/crates/core/src/writer/stats.rs @@ -67,7 +67,6 @@ pub fn create_add( deletion_vector: None, base_row_id: None, default_row_commit_version: None, - stats_parsed: None, clustering_provider: None, }) } @@ -1071,28 +1070,30 @@ mod tests { .join("\n") }); static JSON_ROWS: LazyLock> = LazyLock::new(|| { - std::iter::repeat(json!({ - "meta": { - "kafka": { - "offset": 0, - "partition": 0, - "topic": "some_topic" - }, - "producer": { - "timestamp": "2021-06-22" + std::iter::repeat_n( + json!({ + "meta": { + "kafka": { + "offset": 0, + "partition": 0, + "topic": "some_topic" + }, + "producer": { + "timestamp": "2021-06-22" + }, }, - }, - "some_string": "GET", - "some_int": 302, - "some_bool": true, - "some_list": ["a", "b", "c"], - "some_nested_list": [[42], [84]], - "date": "2021-06-22", - "uuid": "176c770d-92af-4a21-bf76-5d8c5261d659", - })) - .take(100) - .chain( - std::iter::repeat(json!({ + "some_string": "GET", + "some_int": 302, + "some_bool": true, + "some_list": ["a", "b", "c"], + "some_nested_list": [[42], [84]], + "date": "2021-06-22", + "uuid": "176c770d-92af-4a21-bf76-5d8c5261d659", + }), + 100, + ) + .chain(std::iter::repeat_n( + json!({ "meta": { "kafka": { "offset": 100, @@ -1110,11 +1111,11 @@ mod tests { "some_nested_list": [[42], [84]], "date": "2021-06-22", "uuid": "54f3e867-3f7b-4122-a452-9d74fb4fe1ba", - })) - .take(100), - ) - .chain( - std::iter::repeat(json!({ + }), + 100, + )) + .chain(std::iter::repeat_n( + json!({ "meta": { "kafka": { "offset": 0, @@ -1128,9 +1129,9 @@ mod tests { "some_nested_list": [[42], null], "date": "2021-06-22", "uuid": "a98bea04-d119-4f21-8edc-eb218b5849af", - })) - .take(100), - ) + }), + 100, + )) .collect() }); } diff --git a/crates/core/tests/checkpoint_writer.rs b/crates/core/tests/checkpoint_writer.rs index 7239980c32..226918d0ef 100644 --- a/crates/core/tests/checkpoint_writer.rs +++ b/crates/core/tests/checkpoint_writer.rs @@ -64,6 +64,7 @@ mod simple_checkpoint { assert_eq!(12, files.count()); } + #[ignore] #[tokio::test] #[serial] async fn checkpoint_run_length_encoding_test() { @@ -100,7 +101,7 @@ mod simple_checkpoint { // _last_checkpoint should exist and point to the correct version let version = get_last_checkpoint_version(&log_path); - assert_eq!(table.version(), version); + assert_eq!(table.version(), Some(version)); // delta table should load just fine with the checkpoint in place let table_result = deltalake_core::open_table(table_location).await.unwrap(); @@ -222,7 +223,7 @@ mod delete_expired_delta_log_in_checkpoint { checkpoints::create_checkpoint_from_table_uri_and_cleanup( &table.table_uri(), - table.version(), + table.version().unwrap(), None, None, ) @@ -272,7 +273,7 @@ mod delete_expired_delta_log_in_checkpoint { checkpoints::create_checkpoint_from_table_uri_and_cleanup( &table.table_uri(), - table.version(), + table.version().unwrap(), None, None, ) @@ -302,31 +303,15 @@ mod delete_expired_delta_log_in_checkpoint { mod checkpoints_with_tombstones { use super::*; + use ::object_store::path::Path as ObjectStorePath; use chrono::Utc; use deltalake_core::kernel::*; use deltalake_core::table::config::TableProperty; use deltalake_core::*; use maplit::hashmap; - use parquet::file::reader::{FileReader, SerializedFileReader}; - use parquet::schema::types::Type; use pretty_assertions::assert_eq; use std::collections::{HashMap, HashSet}; - use std::fs::File; - use std::iter::FromIterator; - use uuid::Uuid; - - async fn read_checkpoint(path: &str) -> (Type, Vec) { - let file = File::open(path).unwrap(); - let reader = SerializedFileReader::new(file).unwrap(); - let schema = reader.metadata().file_metadata().schema(); - let row_iter = reader.get_row_iter(None).unwrap(); - let mut actions = Vec::new(); - for record in row_iter { - actions.push(Action::from_parquet_record(schema, &record.unwrap()).unwrap()) - } - (schema.clone(), actions) - } #[tokio::test] #[ignore] @@ -360,7 +345,7 @@ mod checkpoints_with_tombstones { table .snapshot() .unwrap() - .all_tombstones(table.object_store().clone()) + .all_tombstones(&table.log_store()) .await .unwrap() .collect::>(), @@ -377,7 +362,7 @@ mod checkpoints_with_tombstones { table .snapshot() .unwrap() - .all_tombstones(table.object_store().clone()) + .all_tombstones(&table.log_store()) .await .unwrap() .count(), @@ -385,68 +370,6 @@ mod checkpoints_with_tombstones { ); // stale removes are deleted from the state } - #[tokio::test] - #[ignore] - async fn test_checkpoint_with_extended_file_metadata_true() { - let path = "../test/tests/data/checkpoints_tombstones/metadata_true"; - let mut table = fs_common::create_table(path, None).await; - let r1 = remove_metadata_true(); - let r2 = remove_metadata_true(); - let version = fs_common::commit_removes(&mut table, vec![&r1, &r2]).await; - let (schema, actions) = create_checkpoint_and_parse(&table, path, version).await; - - assert!(actions.contains(&r1)); - assert!(actions.contains(&r2)); - assert!(schema.contains("size")); - assert!(schema.contains("partitionValues")); - assert!(schema.contains("tags")); - } - - #[tokio::test] - async fn test_checkpoint_with_extended_file_metadata_false() { - let path = "../test/tests/data/checkpoints_tombstones/metadata_false"; - let mut table = fs_common::create_table(path, None).await; - let r1 = remove_metadata_true(); - let r2 = remove_metadata_false(); - let version = fs_common::commit_removes(&mut table, vec![&r1, &r2]).await; - let (schema, actions) = create_checkpoint_and_parse(&table, path, version).await; - - // r2 has extended_file_metadata=false, then every tombstone should be so, even r1 - assert_ne!(actions, vec![r1.clone(), r2.clone()]); - assert!(!schema.contains("size")); - assert!(!schema.contains("partitionValues")); - assert!(!schema.contains("tags")); - let r1_updated = Remove { - extended_file_metadata: Some(false), - size: None, - ..r1 - }; - assert!(actions.contains(&r1_updated)); - assert!(actions.contains(&r2)); - } - - #[tokio::test] - async fn test_checkpoint_with_extended_file_metadata_broken() { - let path = "../test/tests/data/checkpoints_tombstones/metadata_broken"; - let mut table = fs_common::create_table(path, None).await; - let r1 = remove_metadata_broken(); - let r2 = remove_metadata_false(); - let version = fs_common::commit_removes(&mut table, vec![&r1, &r2]).await; - let (schema, actions) = create_checkpoint_and_parse(&table, path, version).await; - - // r1 extended_file_metadata=true, but the size is null. - // We should fix this by setting extended_file_metadata=false - assert!(!schema.contains("size")); - assert!(!schema.contains("partitionValues")); - assert!(!schema.contains("tags")); - assert!(actions.contains(&Remove { - extended_file_metadata: Some(false), - size: None, - ..r1 - })); - assert!(actions.contains(&r2)); - } - async fn pseudo_optimize(table: &mut DeltaTable, offset_millis: i64) -> (HashSet, Add) { let removes: HashSet = table .get_files_iter() @@ -483,68 +406,4 @@ mod checkpoints_with_tombstones { fs_common::commit_actions(table, actions, operation).await; (removes, add) } - - async fn create_checkpoint_and_parse( - table: &DeltaTable, - path: &str, - version: i64, - ) -> (HashSet, Vec) { - checkpoints::create_checkpoint(table, None).await.unwrap(); - let cp_path = format!("{path}/_delta_log/0000000000000000000{version}.checkpoint.parquet"); - let (schema, actions) = read_checkpoint(&cp_path).await; - - let fields = schema - .get_fields() - .iter() - .find(|p| p.name() == "remove") - .unwrap() - .get_fields() - .iter() - .map(|f| f.name().to_string()); - - let actions = actions - .iter() - .filter_map(|a| match a { - Action::Remove(r) => Some(r.clone()), - _ => None, - }) - .collect(); - - (HashSet::from_iter(fields), actions) - } - - // The tags and partition values could be missing, but the size has to present - fn remove_metadata_true() -> Remove { - Remove { - path: Uuid::new_v4().to_string(), - deletion_timestamp: Some(Utc::now().timestamp_millis()), - data_change: false, - extended_file_metadata: Some(true), - partition_values: None, - size: Some(100), - tags: None, - deletion_vector: None, - base_row_id: None, - default_row_commit_version: None, - } - } - - // when metadata is false, then extended fields are null - fn remove_metadata_false() -> Remove { - Remove { - extended_file_metadata: Some(false), - size: None, - partition_values: None, - tags: None, - ..remove_metadata_true() - } - } - - // broken record is when fields are null (the size especially) but metadata is true - fn remove_metadata_broken() -> Remove { - Remove { - extended_file_metadata: Some(true), - ..remove_metadata_false() - } - } } diff --git a/crates/core/tests/command_filesystem_check.rs b/crates/core/tests/command_filesystem_check.rs index fdc1e6fae7..d64b099dcc 100644 --- a/crates/core/tests/command_filesystem_check.rs +++ b/crates/core/tests/command_filesystem_check.rs @@ -25,7 +25,7 @@ async fn test_filesystem_check(context: &IntegrationContext) -> TestResult { let version = table.snapshot()?.version(); let active = table.snapshot()?.files_count(); - // Validate a Dry run does not mutate the table log and indentifies orphaned add actions + // Validate a Dry run does not mutate the table log and identifies orphaned add actions let op = DeltaOps::from(table); let (table, metrics) = op.filesystem_check().with_dry_run(true).await?; assert_eq!(version, table.snapshot()?.version()); @@ -41,7 +41,7 @@ async fn test_filesystem_check(context: &IntegrationContext) -> TestResult { let remove = table .snapshot()? - .all_tombstones(table.object_store().clone()) + .all_tombstones(&table.log_store()) .await? .collect::>(); let remove = remove.get(file).unwrap(); @@ -88,7 +88,7 @@ async fn test_filesystem_check_partitioned() -> TestResult { let remove = table .snapshot()? - .all_tombstones(table.object_store().clone()) + .all_tombstones(&table.log_store()) .await? .collect::>(); let remove = remove.get(file).unwrap(); diff --git a/crates/core/tests/command_merge.rs b/crates/core/tests/command_merge.rs index 7b4c3aad01..c0afc200e2 100644 --- a/crates/core/tests/command_merge.rs +++ b/crates/core/tests/command_merge.rs @@ -7,9 +7,9 @@ use datafusion::dataframe::DataFrame; use datafusion::prelude::SessionContext; use datafusion_common::Column; use datafusion_expr::{col, lit, Expr}; +use deltalake_core::kernel::transaction::TransactionError; use deltalake_core::kernel::{DataType as DeltaDataType, PrimitiveType, StructField, StructType}; use deltalake_core::operations::merge::MergeMetrics; -use deltalake_core::operations::transaction::TransactionError; use deltalake_core::protocol::SaveMode; use deltalake_core::{open_table, DeltaOps, DeltaResult, DeltaTable, DeltaTableError}; use std::sync::Arc; diff --git a/crates/core/tests/command_optimize.rs b/crates/core/tests/command_optimize.rs index 4826647750..4ede21ffba 100644 --- a/crates/core/tests/command_optimize.rs +++ b/crates/core/tests/command_optimize.rs @@ -5,14 +5,14 @@ use arrow_array::{Int32Array, RecordBatch, StringArray}; use arrow_schema::{DataType as ArrowDataType, Field, Schema as ArrowSchema}; use arrow_select::concat::concat_batches; use deltalake_core::errors::DeltaTableError; +use deltalake_core::kernel::transaction::{CommitBuilder, CommitProperties}; use deltalake_core::kernel::{Action, DataType, PrimitiveType, StructField}; +use deltalake_core::logstore::ObjectStoreRef; use deltalake_core::operations::optimize::{ create_merge_plan, MetricDetails, Metrics, OptimizeType, }; -use deltalake_core::operations::transaction::{CommitBuilder, CommitProperties}; use deltalake_core::operations::DeltaOps; use deltalake_core::protocol::DeltaOperation; -use deltalake_core::storage::ObjectStoreRef; use deltalake_core::writer::{DeltaWriter, RecordBatchWriter}; use deltalake_core::{DeltaTable, PartitionFilter, Path}; use futures::TryStreamExt; @@ -168,13 +168,13 @@ async fn test_optimize_non_partitioned_table() -> Result<(), Box> { ) .await?; - let version = dt.version(); + let version = dt.version().unwrap(); assert_eq!(dt.get_files_count(), 5); let optimize = DeltaOps(dt).optimize().with_target_size(2_000_000); let (dt, metrics) = optimize.await?; - assert_eq!(version + 1, dt.version()); + assert_eq!(version + 1, dt.version().unwrap()); assert_eq!(metrics.num_files_added, 1); assert_eq!(metrics.num_files_removed, 4); assert_eq!(metrics.total_considered_files, 5); @@ -232,13 +232,13 @@ async fn test_optimize_with_partitions() -> Result<(), Box> { ) .await?; - let version = dt.version(); + let version = dt.version().unwrap(); let filter = vec![PartitionFilter::try_from(("date", "=", "2022-05-22"))?]; let optimize = DeltaOps(dt).optimize().with_filters(&filter); let (dt, metrics) = optimize.await?; - assert_eq!(version + 1, dt.version()); + assert_eq!(version + 1, dt.version().unwrap()); assert_eq!(metrics.num_files_added, 1); assert_eq!(metrics.num_files_removed, 2); assert_eq!(dt.get_files_count(), 3); @@ -279,7 +279,7 @@ async fn test_conflict_for_remove_actions() -> Result<(), Box> { ) .await?; - let version = dt.version(); + let version = dt.version().unwrap(); //create the merge plan, remove a file, and execute the plan. let filter = vec![PartitionFilter::try_from(("date", "=", "2022-05-22"))?]; @@ -317,7 +317,7 @@ async fn test_conflict_for_remove_actions() -> Result<(), Box> { assert!(maybe_metrics.is_err()); dt.update().await?; - assert_eq!(dt.version(), version + 1); + assert_eq!(dt.version().unwrap(), version + 1); Ok(()) } @@ -342,7 +342,7 @@ async fn test_no_conflict_for_append_actions() -> Result<(), Box> { ) .await?; - let version = dt.version(); + let version = dt.version().unwrap(); let filter = vec![PartitionFilter::try_from(("date", "=", "2022-05-22"))?]; let plan = create_merge_plan( @@ -379,7 +379,7 @@ async fn test_no_conflict_for_append_actions() -> Result<(), Box> { assert_eq!(metrics.num_files_removed, 2); dt.update().await.unwrap(); - assert_eq!(dt.version(), version + 2); + assert_eq!(dt.version().unwrap(), version + 2); Ok(()) } @@ -402,7 +402,7 @@ async fn test_commit_interval() -> Result<(), Box> { } } - let version = dt.version(); + let version = dt.version().unwrap(); let plan = create_merge_plan( OptimizeType::Compact, @@ -428,7 +428,7 @@ async fn test_commit_interval() -> Result<(), Box> { assert_eq!(metrics.num_files_removed, 4); dt.update().await.unwrap(); - assert_eq!(dt.version(), version + 2); + assert_eq!(dt.version().unwrap(), version + 2); Ok(()) } @@ -460,7 +460,7 @@ async fn test_idempotent() -> Result<(), Box> { ) .await?; - let version = dt.version(); + let version = dt.version().unwrap(); let filter = vec![PartitionFilter::try_from(("date", "=", "2022-05-22"))?]; @@ -471,7 +471,7 @@ async fn test_idempotent() -> Result<(), Box> { let (dt, metrics) = optimize.await?; assert_eq!(metrics.num_files_added, 1); assert_eq!(metrics.num_files_removed, 2); - assert_eq!(dt.version(), version + 1); + assert_eq!(dt.version().unwrap(), version + 1); let optimize = DeltaOps(dt) .optimize() @@ -481,7 +481,7 @@ async fn test_idempotent() -> Result<(), Box> { assert_eq!(metrics.num_files_added, 0); assert_eq!(metrics.num_files_removed, 0); - assert_eq!(dt.version(), version + 1); + assert_eq!(dt.version().unwrap(), version + 1); Ok(()) } @@ -569,7 +569,7 @@ async fn test_idempotent_with_multiple_bins() -> Result<(), Box> { ) .await?; - let version = dt.version(); + let version = dt.version().unwrap(); let filter = vec![PartitionFilter::try_from(("date", "=", "2022-05-22"))?]; @@ -580,7 +580,7 @@ async fn test_idempotent_with_multiple_bins() -> Result<(), Box> { let (dt, metrics) = optimize.await?; assert_eq!(metrics.num_files_added, 2); assert_eq!(metrics.num_files_removed, 4); - assert_eq!(dt.version(), version + 1); + assert_eq!(dt.version().unwrap(), version + 1); let optimize = DeltaOps(dt) .optimize() @@ -589,7 +589,7 @@ async fn test_idempotent_with_multiple_bins() -> Result<(), Box> { let (dt, metrics) = optimize.await?; assert_eq!(metrics.num_files_added, 0); assert_eq!(metrics.num_files_removed, 0); - assert_eq!(dt.version(), version + 1); + assert_eq!(dt.version().unwrap(), version + 1); Ok(()) } @@ -615,7 +615,7 @@ async fn test_commit_info() -> Result<(), Box> { ) .await?; - let version = dt.version(); + let version = dt.version().unwrap(); let filter = vec![PartitionFilter::try_from(("date", "=", "2022-05-22"))?]; @@ -872,7 +872,8 @@ async fn read_parquet_file( object_store: ObjectStoreRef, ) -> Result> { let file = object_store.head(path).await?; - let file_reader = ParquetObjectReader::new(object_store, file); + let file_reader = + ParquetObjectReader::new(object_store, file.location).with_file_size(file.size); let batches = ParquetRecordBatchStreamBuilder::new(file_reader) .await? .build()? diff --git a/crates/core/tests/command_restore.rs b/crates/core/tests/command_restore.rs index 6b6bc374ca..6c03c14632 100644 --- a/crates/core/tests/command_restore.rs +++ b/crates/core/tests/command_restore.rs @@ -3,8 +3,8 @@ use arrow_array::{Int32Array, RecordBatch}; use arrow_schema::{DataType as ArrowDataType, Field}; use chrono::DateTime; use deltalake_core::kernel::{DataType, PrimitiveType, StructField}; +use deltalake_core::logstore::commit_uri_from_version; use deltalake_core::protocol::SaveMode; -use deltalake_core::storage::commit_uri_from_version; use deltalake_core::{DeltaOps, DeltaTable}; use itertools::Itertools; use rand::Rng; @@ -179,7 +179,7 @@ async fn test_restore_file_missing() -> Result<(), Box> { for file in context .table .snapshot()? - .all_tombstones(context.table.object_store().clone()) + .all_tombstones(&context.table.log_store()) .await? { let p = tmp_dir.path().join(file.clone().path); @@ -208,7 +208,7 @@ async fn test_restore_allow_file_missing() -> Result<(), Box> { for file in context .table .snapshot()? - .all_tombstones(context.table.object_store().clone()) + .all_tombstones(&context.table.log_store()) .await? { let p = tmp_dir.path().join(file.clone().path); diff --git a/crates/core/tests/command_update_table_metadata.rs b/crates/core/tests/command_update_table_metadata.rs new file mode 100644 index 0000000000..a873e2b67e --- /dev/null +++ b/crates/core/tests/command_update_table_metadata.rs @@ -0,0 +1,439 @@ +use deltalake_core::kernel::transaction::CommitProperties; +use deltalake_core::kernel::StructType; + +use deltalake_core::operations::update_table_metadata::TableMetadataUpdate; +use deltalake_core::operations::DeltaOps; +use serde_json::json; + +/// Basic schema for testing +pub fn get_test_schema() -> StructType { + serde_json::from_value(json!({ + "type": "struct", + "fields": [ + {"name": "id", "type": "integer", "nullable": false, "metadata": {}}, + {"name": "value", "type": "string", "nullable": true, "metadata": {}}, + ] + })) + .unwrap() +} + +#[tokio::test] +async fn test_update_table_name_valid() { + let table = DeltaOps::new_in_memory() + .create() + .with_columns(get_test_schema().fields().cloned()) + .await + .unwrap(); + + let name = "test_table_name"; + let update = TableMetadataUpdate { + name: Some(name.to_string()), + description: None, + }; + let updated_table = DeltaOps(table) + .update_table_metadata() + .with_update(update) + .await + .unwrap(); + + let metadata = updated_table.metadata().unwrap(); + assert_eq!(metadata.name.as_ref().unwrap(), name); + assert_eq!(updated_table.version(), Some(1)); +} + +#[tokio::test] +async fn test_update_table_description_valid() { + let table = DeltaOps::new_in_memory() + .create() + .with_columns(get_test_schema().fields().cloned()) + .await + .unwrap(); + + let description = "This is a test table description"; + let update = TableMetadataUpdate { + name: None, + description: Some(description.to_string()), + }; + let updated_table = DeltaOps(table) + .update_table_metadata() + .with_update(update) + .await + .unwrap(); + + let metadata = updated_table.metadata().unwrap(); + assert_eq!(metadata.description.as_ref().unwrap(), description); + assert_eq!(updated_table.version(), Some(1)); +} + +#[tokio::test] +async fn test_update_table_name_character_limit_valid() { + let table = DeltaOps::new_in_memory() + .create() + .with_columns(get_test_schema().fields().cloned()) + .await + .unwrap(); + + let name_255_chars = "x".repeat(255); + let update = TableMetadataUpdate { + name: Some(name_255_chars.clone()), + description: None, + }; + let updated_table = DeltaOps(table) + .update_table_metadata() + .with_update(update) + .await + .unwrap(); + + let metadata = updated_table.metadata().unwrap(); + assert_eq!(metadata.name.as_ref().unwrap(), &name_255_chars); + assert_eq!(updated_table.version(), Some(1)); +} + +#[tokio::test] +async fn test_update_table_name_character_limit_exceeded() { + let table = DeltaOps::new_in_memory() + .create() + .with_columns(get_test_schema().fields().cloned()) + .await + .unwrap(); + + let name_256_chars = "x".repeat(256); + let update = TableMetadataUpdate { + name: Some(name_256_chars), + description: None, + }; + let result = DeltaOps(table) + .update_table_metadata() + .with_update(update) + .await; + + assert!(result.is_err()); + let error_message = result.unwrap_err().to_string(); + assert!(error_message.contains("Table name cannot be empty and cannot exceed 255 characters")); +} + +#[tokio::test] +async fn test_update_table_description_character_limit_valid() { + let table = DeltaOps::new_in_memory() + .create() + .with_columns(get_test_schema().fields().cloned()) + .await + .unwrap(); + + let description_4000_chars = "y".repeat(4000); + let update = TableMetadataUpdate { + name: None, + description: Some(description_4000_chars.clone()), + }; + let updated_table = DeltaOps(table) + .update_table_metadata() + .with_update(update) + .await + .unwrap(); + + let metadata = updated_table.metadata().unwrap(); + assert_eq!( + metadata.description.as_ref().unwrap(), + &description_4000_chars + ); + assert_eq!(updated_table.version(), Some(1)); +} + +#[tokio::test] +async fn test_update_table_description_character_limit_exceeded() { + let table = DeltaOps::new_in_memory() + .create() + .with_columns(get_test_schema().fields().cloned()) + .await + .unwrap(); + + let description_4001_chars = "y".repeat(4001); + let update = TableMetadataUpdate { + name: None, + description: Some(description_4001_chars), + }; + let result = DeltaOps(table) + .update_table_metadata() + .with_update(update) + .await; + + assert!(result.is_err()); + let error_message = result.unwrap_err().to_string(); + assert!(error_message.contains("Table description cannot exceed 4000 characters")); +} + +#[tokio::test] +async fn test_update_existing_table_name() { + let table = DeltaOps::new_in_memory() + .create() + .with_columns(get_test_schema().fields().cloned()) + .await + .unwrap(); + + let initial_name = "initial_table_name"; + let update = TableMetadataUpdate { + name: Some(initial_name.to_string()), + description: None, + }; + let updated_table = DeltaOps(table) + .update_table_metadata() + .with_update(update) + .await + .unwrap(); + + assert_eq!(updated_table.version(), Some(1)); + assert_eq!( + updated_table.metadata().unwrap().name.as_ref().unwrap(), + initial_name + ); + + let new_name = "updated_table_name"; + let update = TableMetadataUpdate { + name: Some(new_name.to_string()), + description: None, + }; + let final_table = DeltaOps(updated_table) + .update_table_metadata() + .with_update(update) + .await + .unwrap(); + + assert_eq!(final_table.version(), Some(2)); + assert_eq!( + final_table.metadata().unwrap().name.as_ref().unwrap(), + new_name + ); +} + +#[tokio::test] +async fn test_update_existing_table_description() { + let table = DeltaOps::new_in_memory() + .create() + .with_columns(get_test_schema().fields().cloned()) + .await + .unwrap(); + + let initial_description = "Initial table description"; + let update = TableMetadataUpdate { + name: None, + description: Some(initial_description.to_string()), + }; + let updated_table = DeltaOps(table) + .update_table_metadata() + .with_update(update) + .await + .unwrap(); + + assert_eq!(updated_table.version(), Some(1)); + assert_eq!( + updated_table + .metadata() + .unwrap() + .description + .as_ref() + .unwrap(), + initial_description + ); + + let new_description = "Updated table description with more details"; + let update = TableMetadataUpdate { + name: None, + description: Some(new_description.to_string()), + }; + let final_table = DeltaOps(updated_table) + .update_table_metadata() + .with_update(update) + .await + .unwrap(); + + assert_eq!(final_table.version(), Some(2)); + assert_eq!( + final_table + .metadata() + .unwrap() + .description + .as_ref() + .unwrap(), + new_description + ); +} + +#[tokio::test] +async fn test_empty_table_name() { + let table = DeltaOps::new_in_memory() + .create() + .with_columns(get_test_schema().fields().cloned()) + .await + .unwrap(); + + let update = TableMetadataUpdate { + name: Some("".to_string()), + description: None, + }; + let result = DeltaOps(table) + .update_table_metadata() + .with_update(update) + .await; + + assert!(result.is_err()); + let error_message = result.unwrap_err().to_string(); + assert!(error_message.contains("Table name cannot be empty")); +} + +#[tokio::test] +async fn test_empty_table_description() { + let table = DeltaOps::new_in_memory() + .create() + .with_columns(get_test_schema().fields().cloned()) + .await + .unwrap(); + + let update = TableMetadataUpdate { + name: None, + description: Some("".to_string()), + }; + let updated_table = DeltaOps(table) + .update_table_metadata() + .with_update(update) + .await + .unwrap(); + + let metadata = updated_table.metadata().unwrap(); + assert_eq!(metadata.description.as_ref().unwrap(), ""); + assert_eq!(updated_table.version(), Some(1)); +} + +#[tokio::test] +async fn test_no_update_specified() { + let table = DeltaOps::new_in_memory() + .create() + .with_columns(get_test_schema().fields().cloned()) + .await + .unwrap(); + + let result = DeltaOps(table).update_table_metadata().await; + + assert!(result.is_err()); + let error_message = result.unwrap_err().to_string(); + assert!(error_message.contains("No metadata update specified")); +} + +#[tokio::test] +async fn test_with_commit_properties() { + let table = DeltaOps::new_in_memory() + .create() + .with_columns(get_test_schema().fields().cloned()) + .await + .unwrap(); + + let commit_properties = CommitProperties::default().with_metadata([ + ("app_id".to_string(), json!("test-app")), + ("test".to_string(), json!("metadata")), + ]); + + let name = "test_table_with_commit_props"; + let update = TableMetadataUpdate { + name: Some(name.to_string()), + description: None, + }; + let updated_table = DeltaOps(table) + .update_table_metadata() + .with_update(update) + .with_commit_properties(commit_properties) + .await + .unwrap(); + + let metadata = updated_table.metadata().unwrap(); + assert_eq!(metadata.name.as_ref().unwrap(), name); + assert_eq!(updated_table.version(), Some(1)); +} + +#[tokio::test] +async fn test_with_custom_execute_handler() { + use async_trait::async_trait; + use deltalake_core::logstore::LogStoreRef; + use deltalake_core::operations::CustomExecuteHandler; + use deltalake_core::DeltaResult; + use std::sync::Arc; + + #[derive(Debug)] + struct MockExecuteHandler { + pre_execute_called: std::sync::atomic::AtomicBool, + post_execute_called: std::sync::atomic::AtomicBool, + } + + #[async_trait] + impl CustomExecuteHandler for MockExecuteHandler { + async fn pre_execute( + &self, + _log_store: &LogStoreRef, + _operation_id: uuid::Uuid, + ) -> DeltaResult<()> { + self.pre_execute_called + .store(true, std::sync::atomic::Ordering::SeqCst); + Ok(()) + } + + async fn post_execute( + &self, + _log_store: &LogStoreRef, + _operation_id: uuid::Uuid, + ) -> DeltaResult<()> { + self.post_execute_called + .store(true, std::sync::atomic::Ordering::SeqCst); + Ok(()) + } + + async fn before_post_commit_hook( + &self, + _log_store: &LogStoreRef, + _file_operation: bool, + _operation_id: uuid::Uuid, + ) -> DeltaResult<()> { + Ok(()) + } + + async fn after_post_commit_hook( + &self, + _log_store: &LogStoreRef, + _file_operation: bool, + _operation_id: uuid::Uuid, + ) -> DeltaResult<()> { + Ok(()) + } + } + + let table = DeltaOps::new_in_memory() + .create() + .with_columns(get_test_schema().fields().cloned()) + .await + .unwrap(); + + let handler = Arc::new(MockExecuteHandler { + pre_execute_called: std::sync::atomic::AtomicBool::new(false), + post_execute_called: std::sync::atomic::AtomicBool::new(false), + }); + + let name = "test_table_with_handler"; + let update = TableMetadataUpdate { + name: Some(name.to_string()), + description: None, + }; + let updated_table = DeltaOps(table) + .update_table_metadata() + .with_update(update) + .with_custom_execute_handler(handler.clone()) + .await + .unwrap(); + + let metadata = updated_table.metadata().unwrap(); + assert_eq!(metadata.name.as_ref().unwrap(), name); + assert_eq!(updated_table.version(), Some(1)); + + assert!(handler + .pre_execute_called + .load(std::sync::atomic::Ordering::SeqCst)); + assert!(handler + .post_execute_called + .load(std::sync::atomic::Ordering::SeqCst)); +} diff --git a/crates/core/tests/commit_info_format.rs b/crates/core/tests/commit_info_format.rs index df817365b3..1ba00cc9d3 100644 --- a/crates/core/tests/commit_info_format.rs +++ b/crates/core/tests/commit_info_format.rs @@ -1,8 +1,8 @@ #![allow(dead_code)] mod fs_common; +use deltalake_core::kernel::transaction::CommitBuilder; use deltalake_core::kernel::Action; -use deltalake_core::operations::transaction::CommitBuilder; use deltalake_core::protocol::{DeltaOperation, SaveMode}; use serde_json::json; use std::error::Error; diff --git a/crates/core/tests/dat.rs b/crates/core/tests/dat.rs new file mode 100644 index 0000000000..6390a7a271 --- /dev/null +++ b/crates/core/tests/dat.rs @@ -0,0 +1,52 @@ +use std::path::Path; + +use deltalake_core::DeltaTableBuilder; +use deltalake_test::acceptance::read_dat_case; + +static SKIPPED_TESTS: &[&str; 4] = &[ + "iceberg_compat_v1", + "column_mapping", + "check_constraints", + "deletion_vectors", +]; + +fn reader_test_eager(path: &Path) -> datatest_stable::Result<()> { + let root_dir = format!( + "{}/{}", + env!["CARGO_MANIFEST_DIR"], + path.parent().unwrap().to_str().unwrap() + ); + for skipped in SKIPPED_TESTS { + if root_dir.ends_with(skipped) { + println!("Skipping test: {}", skipped); + return Ok(()); + } + } + + tokio::runtime::Builder::new_current_thread() + .enable_all() + .build()? + .block_on(async { + let case = read_dat_case(root_dir).unwrap(); + + let table = DeltaTableBuilder::from_uri(case.table_root().unwrap()) + .load() + .await + .expect("table"); + let table_info = case.table_summary().expect("load summary"); + let snapshot = table.snapshot().expect("Failed to load snapshot"); + let protocol = table.protocol().expect("Failed to load protocol"); + assert_eq!(snapshot.version() as u64, table_info.version); + assert_eq!( + (protocol.min_reader_version, protocol.min_writer_version), + (table_info.min_reader_version, table_info.min_writer_version) + ); + }); + Ok(()) +} + +datatest_stable::harness!( + reader_test_eager, + "../../dat/v0.0.3/reader_tests/generated/", + r"test_case_info\.json" +); diff --git a/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000000.json b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000000.json new file mode 100644 index 0000000000..0316f09771 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000000.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1742317262289,"operation":"CREATE TABLE","operationParameters":{"partitionBy":"[]","clusterBy":"[]","description":null,"isManaged":"false","properties":"{}"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"c0983a7a-aeca-4ba8-a509-d5dbc71a10de"}} +{"metaData":{"id":"5f54ef5f-e511-4114-b6e5-f6c206c068b6","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1742317261939}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":1}} diff --git a/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000001.json b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000001.json new file mode 100644 index 0000000000..23002c6f58 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000001.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317316973,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"a1c49e36-8fcd-46e6-8ede-5b8560f7ec3b"}} +{"add":{"path":"part-00001-665397f5-1435-4478-a598-ca226c99ffcf-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316825,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":0},\"maxValues\":{\"id\":0},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-fd50becf-074e-4a1f-985b-01529e9f7b03-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316820,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1},\"maxValues\":{\"id\":1},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-4fc1f70f-9daa-46e6-83b5-ea8144d4a96d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316822,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":2},\"maxValues\":{\"id\":2},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-efb0808c-3b7f-4a4d-bc36-daa91c074b5b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316825,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":3},\"maxValues\":{\"id\":3},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-c8664e02-01fe-4c2d-8eba-ae84012d7aad-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316825,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":4},\"maxValues\":{\"id\":4},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-7446ef47-3110-4c3f-a2d0-0c71bafc893a-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316820,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":5},\"maxValues\":{\"id\":5},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-ee372cff-6aae-4979-970b-88cc154a31bd-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316822,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":6},\"maxValues\":{\"id\":6},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-9ea59fdf-fc26-4650-a282-9c2cc1906c7c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316823,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":7},\"maxValues\":{\"id\":7},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-d50ebfbf-e534-4bc8-b63d-437f6029da6e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316823,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":8},\"maxValues\":{\"id\":8},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-96e67376-3d15-4895-bd5d-5e0a325bcb83-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316823,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":9},\"maxValues\":{\"id\":9},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000002.json b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000002.json new file mode 100644 index 0000000000..86e3c7470c --- /dev/null +++ b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000002.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317326453,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":1,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"ac59851b-981e-4e82-96ea-36a543cfe254"}} +{"add":{"path":"part-00001-6491d41d-d498-4a89-a291-92d964035606-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":10},\"maxValues\":{\"id\":10},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-d359921a-3cb1-454d-8aa0-ac5c830fcdc5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":11},\"maxValues\":{\"id\":11},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-b2aae64d-1fab-4106-bc87-2454e945dada-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":12},\"maxValues\":{\"id\":12},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-cca989ea-d56e-4e1e-a4ba-538ef7801997-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":13},\"maxValues\":{\"id\":13},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-14706643-f3f0-4ba9-8282-7d55bb4ecacb-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":14},\"maxValues\":{\"id\":14},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-25801ed5-1cf2-43fa-bbdb-8898fc102e64-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":15},\"maxValues\":{\"id\":15},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-03fbeb6f-b8a5-448e-afa7-0f49fca61866-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":16},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-08361c76-870e-4ddf-9153-f67852849ec3-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":17},\"maxValues\":{\"id\":17},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-368e738e-0673-4e76-a1ff-5ba9c755396e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":18},\"maxValues\":{\"id\":18},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-2c9a3837-e2c5-42bd-b888-f3205f4b894c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":19},\"maxValues\":{\"id\":19},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000003.json b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000003.json new file mode 100644 index 0000000000..14665bcd27 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000003.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317330682,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":2,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"5e6fdfee-2a82-40d1-bc83-47b3f0f3f21e"}} +{"add":{"path":"part-00001-b7be8377-b715-4234-b316-201fd2c9c142-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":20},\"maxValues\":{\"id\":20},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-719b1086-b19c-45d1-8c4c-c11db02e2e0b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":21},\"maxValues\":{\"id\":21},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-eb0fd03c-3e47-42ed-9897-e79dd1567fb1-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":22},\"maxValues\":{\"id\":22},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-48b99dee-6d3e-4cfb-b651-4769de7f5b24-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":23},\"maxValues\":{\"id\":23},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-b873a231-8352-4bac-b6f1-b53ee738d212-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330582,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":24},\"maxValues\":{\"id\":24},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-74ed051c-b116-4947-b62c-2086bcd5bb90-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":25},\"maxValues\":{\"id\":25},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-680d9e49-afce-4a7d-bca8-b03438c2fd74-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":26},\"maxValues\":{\"id\":26},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-81c10052-aacc-4ecf-b9cf-64f81b3bd435-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":27},\"maxValues\":{\"id\":27},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-cf9cac69-932c-43bf-8e4b-fd059d519c0f-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":28},\"maxValues\":{\"id\":28},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-43e3252c-8ac7-4c7a-bcb4-15aaf7ae95b9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":29},\"maxValues\":{\"id\":29},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000004.json b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000004.json new file mode 100644 index 0000000000..ed86e283d5 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000004.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317333588,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":3,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"57611e9e-5312-4502-a3ad-c0c78799773e"}} +{"add":{"path":"part-00001-5c92b4bb-af84-4066-8aeb-1e493b7147df-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333524,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":30},\"maxValues\":{\"id\":30},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-b050a084-ab25-420f-bb7b-50eb95d25e4e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333551,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":31},\"maxValues\":{\"id\":31},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-1880b504-fcf5-4f43-92d4-c43e8dd9d7d9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333550,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":32},\"maxValues\":{\"id\":32},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-4af9173b-8472-41cd-8772-e2bdb084c5d5-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317333524,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":33},\"maxValues\":{\"id\":33},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-89cec234-f844-4802-a786-5d9133bbe489-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333523,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":34},\"maxValues\":{\"id\":34},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-fa11b282-ec0d-4513-9baf-2b84c5f94a12-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333550,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":35},\"maxValues\":{\"id\":35},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-895723cb-0dba-4019-a2a9-e6db9a937c91-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333550,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":36},\"maxValues\":{\"id\":36},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-dca57e7a-f859-4b39-bc43-03e1061f1b4e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333551,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":37},\"maxValues\":{\"id\":37},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-96756753-7714-4c07-a238-d5b57f42a8ce-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333523,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":38},\"maxValues\":{\"id\":38},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-9a7b861f-5d9a-41c4-b4ec-7f0d1391acfe-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333551,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":39},\"maxValues\":{\"id\":39},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000005.json b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000005.json new file mode 100644 index 0000000000..f77fd655fe --- /dev/null +++ b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000005.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317336099,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":4,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"203edd74-d2b2-46fe-935a-6222cfb888d4"}} +{"add":{"path":"part-00001-a24fe71c-ba35-47bb-8f3e-636d5991d5ae-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336058,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":40},\"maxValues\":{\"id\":40},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-a56ad50f-bc64-44cb-bb55-e2d177947b3d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336079,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":41},\"maxValues\":{\"id\":41},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-95c53cf1-b472-4c34-b728-1dd7cbed8b2f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336074,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":42},\"maxValues\":{\"id\":42},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-7e442ced-e810-44d9-9d28-3027e652a0ec-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336080,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":43},\"maxValues\":{\"id\":43},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-68248457-2fa3-407e-9de3-759b1e052b99-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336075,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":44},\"maxValues\":{\"id\":44},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-89a48ead-5bf3-4d16-aada-97c11386fcaf-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336076,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":45},\"maxValues\":{\"id\":45},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-42b618de-c46c-4888-9b48-b99493ec2983-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336070,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":46},\"maxValues\":{\"id\":46},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-4dc49305-f4f8-4ec9-9a40-8f4b3bd81324-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336055,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":47},\"maxValues\":{\"id\":47},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-d7bb17a9-223e-474b-9d78-2c745cc35a4b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336054,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":48},\"maxValues\":{\"id\":48},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-68a79bb6-a31e-49bf-848f-2d64ceb834c0-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336079,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":49},\"maxValues\":{\"id\":49},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000006.json b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000006.json new file mode 100644 index 0000000000..28116ba9f2 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000006.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317338700,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":5,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"9ee4b86b-1bbe-4d6e-adbd-8dd4961989fb"}} +{"add":{"path":"part-00001-a8fc5b00-29e4-4a99-961d-b0cbcc23d165-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338678,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":50},\"maxValues\":{\"id\":50},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-5a3079f8-abbc-4b5f-a1e3-340830e59222-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317338670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":51},\"maxValues\":{\"id\":51},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-567d7b20-b6ce-4e96-b500-caa34c80f8a7-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338676,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":52},\"maxValues\":{\"id\":52},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-b794df4b-174f-468a-9de7-2aa865ba7014-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338676,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":53},\"maxValues\":{\"id\":53},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-a15406cf-c141-4f7b-b302-e4b5a145cad5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338675,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":54},\"maxValues\":{\"id\":54},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-8a52349c-d93b-4c59-b493-13486bb5e284-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338680,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":55},\"maxValues\":{\"id\":55},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-1734b4eb-4414-4b3a-8e99-1bd099c9e6b5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":56},\"maxValues\":{\"id\":56},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-02781f2a-6c34-42ca-80a4-e830b2eeb963-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338676,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":57},\"maxValues\":{\"id\":57},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-63ec1c21-c31c-43d4-b5c9-9c206aeeb280-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":58},\"maxValues\":{\"id\":58},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-b732e8e4-7d1b-470d-89a5-86a3f8d8bdc2-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338680,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":59},\"maxValues\":{\"id\":59},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000007.json b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000007.json new file mode 100644 index 0000000000..956c8b508c --- /dev/null +++ b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000007.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317339658,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":6,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"63c885b0-74eb-4075-a02a-a43b8202b3f8"}} +{"add":{"path":"part-00001-f697bc51-b77f-4234-938a-5f85478cedec-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":60},\"maxValues\":{\"id\":60},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-839f210e-cf84-4c5c-b185-fd2fe2b5ee6f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339491,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":61},\"maxValues\":{\"id\":61},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-9ebe1c22-87a1-4f37-a695-77658c3e70a8-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339491,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":62},\"maxValues\":{\"id\":62},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-01b2687c-45e4-484c-b1d6-80e06b5b5d11-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339490,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":63},\"maxValues\":{\"id\":63},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-05662cc4-6a79-4204-aec1-2311a44d8c74-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339511,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":64},\"maxValues\":{\"id\":64},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-67f1b806-ef5f-4f8a-890b-b3b5ad1d234c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339490,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":65},\"maxValues\":{\"id\":65},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-37318455-4128-4e1e-9ab7-5c587ac9fde0-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":66},\"maxValues\":{\"id\":66},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-c5d68934-1f5a-40c4-b5be-1233eb15378a-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339511,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":67},\"maxValues\":{\"id\":67},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-41e40903-13b6-4465-aa3c-bd8cb5e52b18-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":68},\"maxValues\":{\"id\":68},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-110c626e-ea13-4204-8cae-a3183d89a4b7-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":69},\"maxValues\":{\"id\":69},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000008.json b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000008.json new file mode 100644 index 0000000000..ea216c1556 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000008.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317340794,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":7,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"b5902ae9-2aab-46f9-82f9-d68ae45940d7"}} +{"add":{"path":"part-00001-3b62f1d4-2a3e-4611-a55b-e9d2ace11b3c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340752,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":70},\"maxValues\":{\"id\":70},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-9c066923-23f4-45f6-b2af-5a4ecbef1707-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340776,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":71},\"maxValues\":{\"id\":71},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-cfb48b6e-0fc7-4d6b-8ab7-c52f29f71b94-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340752,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":72},\"maxValues\":{\"id\":72},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-b21a45a5-9a53-4dfa-8327-8a82b6b283e9-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317340766,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":73},\"maxValues\":{\"id\":73},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-afa450c4-f649-4c88-817a-6d0bdfc4da6f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340765,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":74},\"maxValues\":{\"id\":74},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-dd96cd25-394d-4873-84e7-f2f6b0eb5a67-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340776,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":75},\"maxValues\":{\"id\":75},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-3a98b7ed-8665-4bc5-8704-6745f7084cd0-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340776,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":76},\"maxValues\":{\"id\":76},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-8dfd3dc5-cf31-42fc-8c55-2ac70ce9e18d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340782,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":77},\"maxValues\":{\"id\":77},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-97e7d287-aee3-445d-a90e-f3b2ef4bd7cd-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340765,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":78},\"maxValues\":{\"id\":78},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-db2340aa-28ff-4826-b39e-07ba516551e9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340779,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":79},\"maxValues\":{\"id\":79},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000009.json b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000009.json new file mode 100644 index 0000000000..1f9d5fb05b --- /dev/null +++ b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000009.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317341714,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":8,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"5379512e-d4d3-42b5-817d-70ecf05f2385"}} +{"add":{"path":"part-00001-4e7175fd-6ffb-4b6a-946c-43aa7c439104-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":80},\"maxValues\":{\"id\":80},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-f3cc4f07-93ec-4a47-add1-b16c1149c3d9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341667,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":81},\"maxValues\":{\"id\":81},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-07581f2d-ee98-4464-a28b-f738e88749e4-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":82},\"maxValues\":{\"id\":82},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-c9d7468d-fc14-445c-8dbd-65d616f8eb05-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341666,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":83},\"maxValues\":{\"id\":83},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-56eb2260-fb49-4138-a5c8-f0ae0949f4e2-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":84},\"maxValues\":{\"id\":84},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-5d15eeca-0fca-4986-a18e-4d86bf5ba2f6-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":85},\"maxValues\":{\"id\":85},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-3e13d733-f55d-42ac-be4f-f4400e999c29-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":86},\"maxValues\":{\"id\":86},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-22c23f1e-26d2-488e-8e07-2de6ae5fded5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341667,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":87},\"maxValues\":{\"id\":87},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-d0debf09-5f82-4c61-8636-27e51fba37e5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341674,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":88},\"maxValues\":{\"id\":88},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-7403e0c4-bb07-4a1b-9fca-a01523713f85-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":89},\"maxValues\":{\"id\":89},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000010.checkpoint.parquet b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000010.checkpoint.parquet new file mode 100644 index 0000000000..9df253f2f7 Binary files /dev/null and b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000010.checkpoint.parquet differ diff --git a/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000010.json b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000010.json new file mode 100644 index 0000000000..c1f8c8b095 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000010.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317342605,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":9,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"ba4b200d-e0ac-4715-ad4e-bed8ef1b20cd"}} +{"add":{"path":"part-00001-55fbea2e-0788-438a-a50c-65f809acc05b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":90},\"maxValues\":{\"id\":90},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-0d891eaf-0e01-46a6-879e-49bbca90c215-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":91},\"maxValues\":{\"id\":91},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-569d12e9-04d5-4fe2-9554-f288047f3386-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342592,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":92},\"maxValues\":{\"id\":92},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-1bc89ec9-8092-49e1-9b1f-123ae50e3d40-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342579,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":93},\"maxValues\":{\"id\":93},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-b4a223d6-1d87-49c9-84c9-a85eece61839-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342591,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":94},\"maxValues\":{\"id\":94},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-4c3b6be7-979c-4f42-8920-efa32b751d97-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":95},\"maxValues\":{\"id\":95},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-6c224359-8995-417a-8b24-b2e530327bc6-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":96},\"maxValues\":{\"id\":96},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-2a40eb21-34d2-48ca-aaa5-55db674f56de-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342590,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":97},\"maxValues\":{\"id\":97},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-ca8256ed-98cd-460d-8de2-9f6f7f388703-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342579,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":98},\"maxValues\":{\"id\":98},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-f131fc78-c201-4e8d-b194-222b2e79778d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":99},\"maxValues\":{\"id\":99},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000011.json b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000011.json new file mode 100644 index 0000000000..05863f6cd0 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000011.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317349152,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":10,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"1657fc30-879d-4b0b-972a-4e3a079fdd7a"}} +{"add":{"path":"part-00001-ceaadd5e-615b-455d-8f4b-052b9c94c7b6-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349136,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":100},\"maxValues\":{\"id\":100},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-b7dba1e7-b1e5-4f02-a223-69ec7353ab45-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349123,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":101},\"maxValues\":{\"id\":101},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-f099cf4d-d418-4852-8580-091908847a66-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317349122,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":102},\"maxValues\":{\"id\":102},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-418e8d25-7316-442a-9bc8-616ed01231eb-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349136,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":103},\"maxValues\":{\"id\":103},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-7538a9c2-1ccb-4150-b162-ef8d826fe30f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349123,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":104},\"maxValues\":{\"id\":104},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-7619f42c-5bc4-4e77-b037-f36481c8b63c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349123,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":105},\"maxValues\":{\"id\":105},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-eb49d85f-91cc-4293-9339-a664ee905b0f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349134,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":106},\"maxValues\":{\"id\":106},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-81e22719-7705-4703-b2dd-c4e2982217a7-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349122,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":107},\"maxValues\":{\"id\":107},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-883858d1-9df6-4b55-a2be-5b8387134617-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349122,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":108},\"maxValues\":{\"id\":108},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-07d91938-ac89-48cc-a657-6067d2d9f67e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349137,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":109},\"maxValues\":{\"id\":109},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000012.json b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000012.json new file mode 100644 index 0000000000..4cc44fa8e8 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000012.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317349950,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":11,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"9a035bdd-f892-4449-9c39-401f31fcada6"}} +{"add":{"path":"part-00001-f3b19100-b5b3-4e72-8658-7a937e9ed515-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349924,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":110},\"maxValues\":{\"id\":110},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-54f2324a-e97f-4def-9101-9cc10599ba06-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349919,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":111},\"maxValues\":{\"id\":111},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-3f7ca40a-6497-4208-8a1a-11062456a5a9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349936,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":112},\"maxValues\":{\"id\":112},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-a14852b2-c743-4a4a-b9c1-0c9472c51699-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349929,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":113},\"maxValues\":{\"id\":113},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-97d06207-5584-43df-afc2-2d1738d79193-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349943,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":114},\"maxValues\":{\"id\":114},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-0d431f03-6dbf-40e7-96fc-b1ebbbe9fc65-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349922,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":115},\"maxValues\":{\"id\":115},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-af0f0232-33c8-4315-821b-8bb1323b7a26-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317349936,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":116},\"maxValues\":{\"id\":116},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-4f744428-d088-497e-afd3-0b374e453e7c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349936,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":117},\"maxValues\":{\"id\":117},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-694064b8-137e-45cd-b2ea-e28af172a2dc-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349918,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":118},\"maxValues\":{\"id\":118},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-56073753-4c1c-4a68-9b4a-13ef5d1a75fb-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349938,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":119},\"maxValues\":{\"id\":119},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000013.json b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000013.json new file mode 100644 index 0000000000..b2d03d3ead --- /dev/null +++ b/crates/core/tests/data_err_logs/table_a/_delta_log/00000000000000000013.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317350712,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":12,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"c3cd0fa1-9c72-4344-8225-0b787e52d5e0"}} +{"add":{"path":"part-00001-7a0d95f8-e122-4cf6-b89c-389036a9b415-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350696,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":120},\"maxValues\":{\"id\":120},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-f1f035c1-bf0f-485c-950d-c81d0d2aa8a2-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350706,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":121},\"maxValues\":{\"id\":121},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-d7a51e45-70f3-4379-819b-341951abefff-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350691,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":122},\"maxValues\":{\"id\":122},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-4828722c-5799-4be1-ace1-14bd7f477dbf-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350691,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":123},\"maxValues\":{\"id\":123},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-f3c3c72e-5d71-4dc9-9e15-342f1d6cb6cc-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350701,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":124},\"maxValues\":{\"id\":124},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-e8d74ede-8876-4f55-8e9f-1bbde0d07a35-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350696,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":125},\"maxValues\":{\"id\":125},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-91052146-2292-45c3-b57e-1fd2dd6be6ed-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350692,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":126},\"maxValues\":{\"id\":126},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-9d203964-9f4c-4c84-ad77-9ba305bb6572-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350706,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":127},\"maxValues\":{\"id\":127},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-8362228b-acf6-4937-875b-26c013c342e1-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350690,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":128},\"maxValues\":{\"id\":128},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-6f57658e-1953-4b59-b504-27c9e8c5cc3b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350677,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":129},\"maxValues\":{\"id\":129},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_a/_delta_log/_last_checkpoint b/crates/core/tests/data_err_logs/table_a/_delta_log/_last_checkpoint new file mode 100644 index 0000000000..604c2b31a2 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_a/_delta_log/_last_checkpoint @@ -0,0 +1 @@ +{"version":10,"size":102,"sizeInBytes":21615,"numOfAddFiles":100,"checkpointSchema":{"type":"struct","fields":[{"name":"txn","type":{"type":"struct","fields":[{"name":"appId","type":"string","nullable":true,"metadata":{}},{"name":"version","type":"long","nullable":true,"metadata":{}},{"name":"lastUpdated","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"add","type":{"type":"struct","fields":[{"name":"path","type":"string","nullable":true,"metadata":{}},{"name":"partitionValues","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"size","type":"long","nullable":true,"metadata":{}},{"name":"modificationTime","type":"long","nullable":true,"metadata":{}},{"name":"dataChange","type":"boolean","nullable":true,"metadata":{}},{"name":"tags","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"deletionVector","type":{"type":"struct","fields":[{"name":"storageType","type":"string","nullable":true,"metadata":{}},{"name":"pathOrInlineDv","type":"string","nullable":true,"metadata":{}},{"name":"offset","type":"integer","nullable":true,"metadata":{}},{"name":"sizeInBytes","type":"integer","nullable":true,"metadata":{}},{"name":"cardinality","type":"long","nullable":true,"metadata":{}},{"name":"maxRowIndex","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"baseRowId","type":"long","nullable":true,"metadata":{}},{"name":"defaultRowCommitVersion","type":"long","nullable":true,"metadata":{}},{"name":"clusteringProvider","type":"string","nullable":true,"metadata":{}},{"name":"stats","type":"string","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"remove","type":{"type":"struct","fields":[{"name":"path","type":"string","nullable":true,"metadata":{}},{"name":"deletionTimestamp","type":"long","nullable":true,"metadata":{}},{"name":"dataChange","type":"boolean","nullable":true,"metadata":{}},{"name":"extendedFileMetadata","type":"boolean","nullable":true,"metadata":{}},{"name":"partitionValues","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"size","type":"long","nullable":true,"metadata":{}},{"name":"deletionVector","type":{"type":"struct","fields":[{"name":"storageType","type":"string","nullable":true,"metadata":{}},{"name":"pathOrInlineDv","type":"string","nullable":true,"metadata":{}},{"name":"offset","type":"integer","nullable":true,"metadata":{}},{"name":"sizeInBytes","type":"integer","nullable":true,"metadata":{}},{"name":"cardinality","type":"long","nullable":true,"metadata":{}},{"name":"maxRowIndex","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"baseRowId","type":"long","nullable":true,"metadata":{}},{"name":"defaultRowCommitVersion","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"metaData","type":{"type":"struct","fields":[{"name":"id","type":"string","nullable":true,"metadata":{}},{"name":"name","type":"string","nullable":true,"metadata":{}},{"name":"description","type":"string","nullable":true,"metadata":{}},{"name":"format","type":{"type":"struct","fields":[{"name":"provider","type":"string","nullable":true,"metadata":{}},{"name":"options","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"schemaString","type":"string","nullable":true,"metadata":{}},{"name":"partitionColumns","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"configuration","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"createdTime","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"protocol","type":{"type":"struct","fields":[{"name":"minReaderVersion","type":"integer","nullable":true,"metadata":{}},{"name":"minWriterVersion","type":"integer","nullable":true,"metadata":{}},{"name":"readerFeatures","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"writerFeatures","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"domainMetadata","type":{"type":"struct","fields":[{"name":"domain","type":"string","nullable":true,"metadata":{}},{"name":"configuration","type":"string","nullable":true,"metadata":{}},{"name":"removed","type":"boolean","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}}]},"checksum":"94a578f92841fa7ba9cdee96b5905fdb"} diff --git a/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000000.json b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000000.json new file mode 100644 index 0000000000..0316f09771 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000000.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1742317262289,"operation":"CREATE TABLE","operationParameters":{"partitionBy":"[]","clusterBy":"[]","description":null,"isManaged":"false","properties":"{}"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"c0983a7a-aeca-4ba8-a509-d5dbc71a10de"}} +{"metaData":{"id":"5f54ef5f-e511-4114-b6e5-f6c206c068b6","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1742317261939}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":1}} diff --git a/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000001.json b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000001.json new file mode 100644 index 0000000000..23002c6f58 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000001.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317316973,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"a1c49e36-8fcd-46e6-8ede-5b8560f7ec3b"}} +{"add":{"path":"part-00001-665397f5-1435-4478-a598-ca226c99ffcf-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316825,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":0},\"maxValues\":{\"id\":0},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-fd50becf-074e-4a1f-985b-01529e9f7b03-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316820,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1},\"maxValues\":{\"id\":1},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-4fc1f70f-9daa-46e6-83b5-ea8144d4a96d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316822,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":2},\"maxValues\":{\"id\":2},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-efb0808c-3b7f-4a4d-bc36-daa91c074b5b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316825,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":3},\"maxValues\":{\"id\":3},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-c8664e02-01fe-4c2d-8eba-ae84012d7aad-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316825,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":4},\"maxValues\":{\"id\":4},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-7446ef47-3110-4c3f-a2d0-0c71bafc893a-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316820,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":5},\"maxValues\":{\"id\":5},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-ee372cff-6aae-4979-970b-88cc154a31bd-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316822,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":6},\"maxValues\":{\"id\":6},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-9ea59fdf-fc26-4650-a282-9c2cc1906c7c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316823,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":7},\"maxValues\":{\"id\":7},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-d50ebfbf-e534-4bc8-b63d-437f6029da6e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316823,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":8},\"maxValues\":{\"id\":8},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-96e67376-3d15-4895-bd5d-5e0a325bcb83-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316823,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":9},\"maxValues\":{\"id\":9},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000002.json b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000002.json new file mode 100644 index 0000000000..86e3c7470c --- /dev/null +++ b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000002.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317326453,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":1,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"ac59851b-981e-4e82-96ea-36a543cfe254"}} +{"add":{"path":"part-00001-6491d41d-d498-4a89-a291-92d964035606-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":10},\"maxValues\":{\"id\":10},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-d359921a-3cb1-454d-8aa0-ac5c830fcdc5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":11},\"maxValues\":{\"id\":11},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-b2aae64d-1fab-4106-bc87-2454e945dada-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":12},\"maxValues\":{\"id\":12},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-cca989ea-d56e-4e1e-a4ba-538ef7801997-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":13},\"maxValues\":{\"id\":13},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-14706643-f3f0-4ba9-8282-7d55bb4ecacb-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":14},\"maxValues\":{\"id\":14},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-25801ed5-1cf2-43fa-bbdb-8898fc102e64-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":15},\"maxValues\":{\"id\":15},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-03fbeb6f-b8a5-448e-afa7-0f49fca61866-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":16},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-08361c76-870e-4ddf-9153-f67852849ec3-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":17},\"maxValues\":{\"id\":17},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-368e738e-0673-4e76-a1ff-5ba9c755396e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":18},\"maxValues\":{\"id\":18},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-2c9a3837-e2c5-42bd-b888-f3205f4b894c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":19},\"maxValues\":{\"id\":19},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000003.json b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000003.json new file mode 100644 index 0000000000..14665bcd27 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000003.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317330682,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":2,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"5e6fdfee-2a82-40d1-bc83-47b3f0f3f21e"}} +{"add":{"path":"part-00001-b7be8377-b715-4234-b316-201fd2c9c142-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":20},\"maxValues\":{\"id\":20},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-719b1086-b19c-45d1-8c4c-c11db02e2e0b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":21},\"maxValues\":{\"id\":21},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-eb0fd03c-3e47-42ed-9897-e79dd1567fb1-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":22},\"maxValues\":{\"id\":22},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-48b99dee-6d3e-4cfb-b651-4769de7f5b24-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":23},\"maxValues\":{\"id\":23},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-b873a231-8352-4bac-b6f1-b53ee738d212-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330582,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":24},\"maxValues\":{\"id\":24},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-74ed051c-b116-4947-b62c-2086bcd5bb90-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":25},\"maxValues\":{\"id\":25},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-680d9e49-afce-4a7d-bca8-b03438c2fd74-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":26},\"maxValues\":{\"id\":26},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-81c10052-aacc-4ecf-b9cf-64f81b3bd435-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":27},\"maxValues\":{\"id\":27},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-cf9cac69-932c-43bf-8e4b-fd059d519c0f-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":28},\"maxValues\":{\"id\":28},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-43e3252c-8ac7-4c7a-bcb4-15aaf7ae95b9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":29},\"maxValues\":{\"id\":29},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000004.json b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000004.json new file mode 100644 index 0000000000..ed86e283d5 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000004.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317333588,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":3,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"57611e9e-5312-4502-a3ad-c0c78799773e"}} +{"add":{"path":"part-00001-5c92b4bb-af84-4066-8aeb-1e493b7147df-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333524,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":30},\"maxValues\":{\"id\":30},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-b050a084-ab25-420f-bb7b-50eb95d25e4e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333551,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":31},\"maxValues\":{\"id\":31},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-1880b504-fcf5-4f43-92d4-c43e8dd9d7d9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333550,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":32},\"maxValues\":{\"id\":32},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-4af9173b-8472-41cd-8772-e2bdb084c5d5-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317333524,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":33},\"maxValues\":{\"id\":33},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-89cec234-f844-4802-a786-5d9133bbe489-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333523,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":34},\"maxValues\":{\"id\":34},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-fa11b282-ec0d-4513-9baf-2b84c5f94a12-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333550,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":35},\"maxValues\":{\"id\":35},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-895723cb-0dba-4019-a2a9-e6db9a937c91-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333550,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":36},\"maxValues\":{\"id\":36},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-dca57e7a-f859-4b39-bc43-03e1061f1b4e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333551,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":37},\"maxValues\":{\"id\":37},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-96756753-7714-4c07-a238-d5b57f42a8ce-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333523,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":38},\"maxValues\":{\"id\":38},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-9a7b861f-5d9a-41c4-b4ec-7f0d1391acfe-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333551,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":39},\"maxValues\":{\"id\":39},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000005.json b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000005.json new file mode 100644 index 0000000000..f77fd655fe --- /dev/null +++ b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000005.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317336099,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":4,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"203edd74-d2b2-46fe-935a-6222cfb888d4"}} +{"add":{"path":"part-00001-a24fe71c-ba35-47bb-8f3e-636d5991d5ae-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336058,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":40},\"maxValues\":{\"id\":40},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-a56ad50f-bc64-44cb-bb55-e2d177947b3d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336079,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":41},\"maxValues\":{\"id\":41},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-95c53cf1-b472-4c34-b728-1dd7cbed8b2f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336074,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":42},\"maxValues\":{\"id\":42},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-7e442ced-e810-44d9-9d28-3027e652a0ec-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336080,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":43},\"maxValues\":{\"id\":43},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-68248457-2fa3-407e-9de3-759b1e052b99-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336075,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":44},\"maxValues\":{\"id\":44},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-89a48ead-5bf3-4d16-aada-97c11386fcaf-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336076,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":45},\"maxValues\":{\"id\":45},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-42b618de-c46c-4888-9b48-b99493ec2983-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336070,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":46},\"maxValues\":{\"id\":46},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-4dc49305-f4f8-4ec9-9a40-8f4b3bd81324-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336055,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":47},\"maxValues\":{\"id\":47},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-d7bb17a9-223e-474b-9d78-2c745cc35a4b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336054,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":48},\"maxValues\":{\"id\":48},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-68a79bb6-a31e-49bf-848f-2d64ceb834c0-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336079,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":49},\"maxValues\":{\"id\":49},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000006.json b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000006.json new file mode 100644 index 0000000000..28116ba9f2 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000006.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317338700,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":5,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"9ee4b86b-1bbe-4d6e-adbd-8dd4961989fb"}} +{"add":{"path":"part-00001-a8fc5b00-29e4-4a99-961d-b0cbcc23d165-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338678,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":50},\"maxValues\":{\"id\":50},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-5a3079f8-abbc-4b5f-a1e3-340830e59222-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317338670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":51},\"maxValues\":{\"id\":51},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-567d7b20-b6ce-4e96-b500-caa34c80f8a7-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338676,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":52},\"maxValues\":{\"id\":52},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-b794df4b-174f-468a-9de7-2aa865ba7014-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338676,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":53},\"maxValues\":{\"id\":53},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-a15406cf-c141-4f7b-b302-e4b5a145cad5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338675,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":54},\"maxValues\":{\"id\":54},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-8a52349c-d93b-4c59-b493-13486bb5e284-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338680,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":55},\"maxValues\":{\"id\":55},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-1734b4eb-4414-4b3a-8e99-1bd099c9e6b5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":56},\"maxValues\":{\"id\":56},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-02781f2a-6c34-42ca-80a4-e830b2eeb963-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338676,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":57},\"maxValues\":{\"id\":57},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-63ec1c21-c31c-43d4-b5c9-9c206aeeb280-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":58},\"maxValues\":{\"id\":58},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-b732e8e4-7d1b-470d-89a5-86a3f8d8bdc2-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338680,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":59},\"maxValues\":{\"id\":59},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000008.json b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000008.json new file mode 100644 index 0000000000..ea216c1556 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000008.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317340794,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":7,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"b5902ae9-2aab-46f9-82f9-d68ae45940d7"}} +{"add":{"path":"part-00001-3b62f1d4-2a3e-4611-a55b-e9d2ace11b3c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340752,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":70},\"maxValues\":{\"id\":70},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-9c066923-23f4-45f6-b2af-5a4ecbef1707-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340776,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":71},\"maxValues\":{\"id\":71},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-cfb48b6e-0fc7-4d6b-8ab7-c52f29f71b94-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340752,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":72},\"maxValues\":{\"id\":72},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-b21a45a5-9a53-4dfa-8327-8a82b6b283e9-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317340766,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":73},\"maxValues\":{\"id\":73},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-afa450c4-f649-4c88-817a-6d0bdfc4da6f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340765,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":74},\"maxValues\":{\"id\":74},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-dd96cd25-394d-4873-84e7-f2f6b0eb5a67-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340776,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":75},\"maxValues\":{\"id\":75},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-3a98b7ed-8665-4bc5-8704-6745f7084cd0-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340776,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":76},\"maxValues\":{\"id\":76},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-8dfd3dc5-cf31-42fc-8c55-2ac70ce9e18d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340782,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":77},\"maxValues\":{\"id\":77},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-97e7d287-aee3-445d-a90e-f3b2ef4bd7cd-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340765,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":78},\"maxValues\":{\"id\":78},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-db2340aa-28ff-4826-b39e-07ba516551e9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340779,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":79},\"maxValues\":{\"id\":79},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000009.json b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000009.json new file mode 100644 index 0000000000..1f9d5fb05b --- /dev/null +++ b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000009.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317341714,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":8,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"5379512e-d4d3-42b5-817d-70ecf05f2385"}} +{"add":{"path":"part-00001-4e7175fd-6ffb-4b6a-946c-43aa7c439104-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":80},\"maxValues\":{\"id\":80},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-f3cc4f07-93ec-4a47-add1-b16c1149c3d9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341667,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":81},\"maxValues\":{\"id\":81},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-07581f2d-ee98-4464-a28b-f738e88749e4-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":82},\"maxValues\":{\"id\":82},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-c9d7468d-fc14-445c-8dbd-65d616f8eb05-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341666,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":83},\"maxValues\":{\"id\":83},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-56eb2260-fb49-4138-a5c8-f0ae0949f4e2-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":84},\"maxValues\":{\"id\":84},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-5d15eeca-0fca-4986-a18e-4d86bf5ba2f6-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":85},\"maxValues\":{\"id\":85},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-3e13d733-f55d-42ac-be4f-f4400e999c29-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":86},\"maxValues\":{\"id\":86},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-22c23f1e-26d2-488e-8e07-2de6ae5fded5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341667,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":87},\"maxValues\":{\"id\":87},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-d0debf09-5f82-4c61-8636-27e51fba37e5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341674,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":88},\"maxValues\":{\"id\":88},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-7403e0c4-bb07-4a1b-9fca-a01523713f85-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":89},\"maxValues\":{\"id\":89},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000010.checkpoint.parquet b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000010.checkpoint.parquet new file mode 100644 index 0000000000..9df253f2f7 Binary files /dev/null and b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000010.checkpoint.parquet differ diff --git a/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000010.json b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000010.json new file mode 100644 index 0000000000..c1f8c8b095 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000010.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317342605,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":9,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"ba4b200d-e0ac-4715-ad4e-bed8ef1b20cd"}} +{"add":{"path":"part-00001-55fbea2e-0788-438a-a50c-65f809acc05b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":90},\"maxValues\":{\"id\":90},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-0d891eaf-0e01-46a6-879e-49bbca90c215-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":91},\"maxValues\":{\"id\":91},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-569d12e9-04d5-4fe2-9554-f288047f3386-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342592,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":92},\"maxValues\":{\"id\":92},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-1bc89ec9-8092-49e1-9b1f-123ae50e3d40-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342579,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":93},\"maxValues\":{\"id\":93},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-b4a223d6-1d87-49c9-84c9-a85eece61839-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342591,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":94},\"maxValues\":{\"id\":94},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-4c3b6be7-979c-4f42-8920-efa32b751d97-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":95},\"maxValues\":{\"id\":95},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-6c224359-8995-417a-8b24-b2e530327bc6-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":96},\"maxValues\":{\"id\":96},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-2a40eb21-34d2-48ca-aaa5-55db674f56de-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342590,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":97},\"maxValues\":{\"id\":97},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-ca8256ed-98cd-460d-8de2-9f6f7f388703-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342579,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":98},\"maxValues\":{\"id\":98},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-f131fc78-c201-4e8d-b194-222b2e79778d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":99},\"maxValues\":{\"id\":99},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000011.json b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000011.json new file mode 100644 index 0000000000..05863f6cd0 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000011.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317349152,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":10,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"1657fc30-879d-4b0b-972a-4e3a079fdd7a"}} +{"add":{"path":"part-00001-ceaadd5e-615b-455d-8f4b-052b9c94c7b6-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349136,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":100},\"maxValues\":{\"id\":100},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-b7dba1e7-b1e5-4f02-a223-69ec7353ab45-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349123,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":101},\"maxValues\":{\"id\":101},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-f099cf4d-d418-4852-8580-091908847a66-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317349122,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":102},\"maxValues\":{\"id\":102},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-418e8d25-7316-442a-9bc8-616ed01231eb-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349136,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":103},\"maxValues\":{\"id\":103},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-7538a9c2-1ccb-4150-b162-ef8d826fe30f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349123,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":104},\"maxValues\":{\"id\":104},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-7619f42c-5bc4-4e77-b037-f36481c8b63c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349123,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":105},\"maxValues\":{\"id\":105},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-eb49d85f-91cc-4293-9339-a664ee905b0f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349134,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":106},\"maxValues\":{\"id\":106},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-81e22719-7705-4703-b2dd-c4e2982217a7-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349122,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":107},\"maxValues\":{\"id\":107},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-883858d1-9df6-4b55-a2be-5b8387134617-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349122,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":108},\"maxValues\":{\"id\":108},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-07d91938-ac89-48cc-a657-6067d2d9f67e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349137,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":109},\"maxValues\":{\"id\":109},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000012.json b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000012.json new file mode 100644 index 0000000000..4cc44fa8e8 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000012.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317349950,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":11,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"9a035bdd-f892-4449-9c39-401f31fcada6"}} +{"add":{"path":"part-00001-f3b19100-b5b3-4e72-8658-7a937e9ed515-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349924,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":110},\"maxValues\":{\"id\":110},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-54f2324a-e97f-4def-9101-9cc10599ba06-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349919,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":111},\"maxValues\":{\"id\":111},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-3f7ca40a-6497-4208-8a1a-11062456a5a9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349936,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":112},\"maxValues\":{\"id\":112},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-a14852b2-c743-4a4a-b9c1-0c9472c51699-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349929,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":113},\"maxValues\":{\"id\":113},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-97d06207-5584-43df-afc2-2d1738d79193-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349943,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":114},\"maxValues\":{\"id\":114},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-0d431f03-6dbf-40e7-96fc-b1ebbbe9fc65-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349922,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":115},\"maxValues\":{\"id\":115},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-af0f0232-33c8-4315-821b-8bb1323b7a26-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317349936,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":116},\"maxValues\":{\"id\":116},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-4f744428-d088-497e-afd3-0b374e453e7c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349936,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":117},\"maxValues\":{\"id\":117},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-694064b8-137e-45cd-b2ea-e28af172a2dc-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349918,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":118},\"maxValues\":{\"id\":118},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-56073753-4c1c-4a68-9b4a-13ef5d1a75fb-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349938,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":119},\"maxValues\":{\"id\":119},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000013.json b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000013.json new file mode 100644 index 0000000000..b2d03d3ead --- /dev/null +++ b/crates/core/tests/data_err_logs/table_b/_delta_log/00000000000000000013.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317350712,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":12,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"c3cd0fa1-9c72-4344-8225-0b787e52d5e0"}} +{"add":{"path":"part-00001-7a0d95f8-e122-4cf6-b89c-389036a9b415-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350696,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":120},\"maxValues\":{\"id\":120},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-f1f035c1-bf0f-485c-950d-c81d0d2aa8a2-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350706,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":121},\"maxValues\":{\"id\":121},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-d7a51e45-70f3-4379-819b-341951abefff-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350691,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":122},\"maxValues\":{\"id\":122},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-4828722c-5799-4be1-ace1-14bd7f477dbf-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350691,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":123},\"maxValues\":{\"id\":123},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-f3c3c72e-5d71-4dc9-9e15-342f1d6cb6cc-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350701,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":124},\"maxValues\":{\"id\":124},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-e8d74ede-8876-4f55-8e9f-1bbde0d07a35-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350696,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":125},\"maxValues\":{\"id\":125},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-91052146-2292-45c3-b57e-1fd2dd6be6ed-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350692,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":126},\"maxValues\":{\"id\":126},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-9d203964-9f4c-4c84-ad77-9ba305bb6572-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350706,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":127},\"maxValues\":{\"id\":127},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-8362228b-acf6-4937-875b-26c013c342e1-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350690,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":128},\"maxValues\":{\"id\":128},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-6f57658e-1953-4b59-b504-27c9e8c5cc3b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350677,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":129},\"maxValues\":{\"id\":129},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_b/_delta_log/_last_checkpoint b/crates/core/tests/data_err_logs/table_b/_delta_log/_last_checkpoint new file mode 100644 index 0000000000..604c2b31a2 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_b/_delta_log/_last_checkpoint @@ -0,0 +1 @@ +{"version":10,"size":102,"sizeInBytes":21615,"numOfAddFiles":100,"checkpointSchema":{"type":"struct","fields":[{"name":"txn","type":{"type":"struct","fields":[{"name":"appId","type":"string","nullable":true,"metadata":{}},{"name":"version","type":"long","nullable":true,"metadata":{}},{"name":"lastUpdated","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"add","type":{"type":"struct","fields":[{"name":"path","type":"string","nullable":true,"metadata":{}},{"name":"partitionValues","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"size","type":"long","nullable":true,"metadata":{}},{"name":"modificationTime","type":"long","nullable":true,"metadata":{}},{"name":"dataChange","type":"boolean","nullable":true,"metadata":{}},{"name":"tags","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"deletionVector","type":{"type":"struct","fields":[{"name":"storageType","type":"string","nullable":true,"metadata":{}},{"name":"pathOrInlineDv","type":"string","nullable":true,"metadata":{}},{"name":"offset","type":"integer","nullable":true,"metadata":{}},{"name":"sizeInBytes","type":"integer","nullable":true,"metadata":{}},{"name":"cardinality","type":"long","nullable":true,"metadata":{}},{"name":"maxRowIndex","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"baseRowId","type":"long","nullable":true,"metadata":{}},{"name":"defaultRowCommitVersion","type":"long","nullable":true,"metadata":{}},{"name":"clusteringProvider","type":"string","nullable":true,"metadata":{}},{"name":"stats","type":"string","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"remove","type":{"type":"struct","fields":[{"name":"path","type":"string","nullable":true,"metadata":{}},{"name":"deletionTimestamp","type":"long","nullable":true,"metadata":{}},{"name":"dataChange","type":"boolean","nullable":true,"metadata":{}},{"name":"extendedFileMetadata","type":"boolean","nullable":true,"metadata":{}},{"name":"partitionValues","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"size","type":"long","nullable":true,"metadata":{}},{"name":"deletionVector","type":{"type":"struct","fields":[{"name":"storageType","type":"string","nullable":true,"metadata":{}},{"name":"pathOrInlineDv","type":"string","nullable":true,"metadata":{}},{"name":"offset","type":"integer","nullable":true,"metadata":{}},{"name":"sizeInBytes","type":"integer","nullable":true,"metadata":{}},{"name":"cardinality","type":"long","nullable":true,"metadata":{}},{"name":"maxRowIndex","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"baseRowId","type":"long","nullable":true,"metadata":{}},{"name":"defaultRowCommitVersion","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"metaData","type":{"type":"struct","fields":[{"name":"id","type":"string","nullable":true,"metadata":{}},{"name":"name","type":"string","nullable":true,"metadata":{}},{"name":"description","type":"string","nullable":true,"metadata":{}},{"name":"format","type":{"type":"struct","fields":[{"name":"provider","type":"string","nullable":true,"metadata":{}},{"name":"options","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"schemaString","type":"string","nullable":true,"metadata":{}},{"name":"partitionColumns","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"configuration","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"createdTime","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"protocol","type":{"type":"struct","fields":[{"name":"minReaderVersion","type":"integer","nullable":true,"metadata":{}},{"name":"minWriterVersion","type":"integer","nullable":true,"metadata":{}},{"name":"readerFeatures","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"writerFeatures","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"domainMetadata","type":{"type":"struct","fields":[{"name":"domain","type":"string","nullable":true,"metadata":{}},{"name":"configuration","type":"string","nullable":true,"metadata":{}},{"name":"removed","type":"boolean","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}}]},"checksum":"94a578f92841fa7ba9cdee96b5905fdb"} diff --git a/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000000.json b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000000.json new file mode 100644 index 0000000000..0316f09771 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000000.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1742317262289,"operation":"CREATE TABLE","operationParameters":{"partitionBy":"[]","clusterBy":"[]","description":null,"isManaged":"false","properties":"{}"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"c0983a7a-aeca-4ba8-a509-d5dbc71a10de"}} +{"metaData":{"id":"5f54ef5f-e511-4114-b6e5-f6c206c068b6","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1742317261939}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":1}} diff --git a/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000001.json b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000001.json new file mode 100644 index 0000000000..23002c6f58 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000001.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317316973,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"a1c49e36-8fcd-46e6-8ede-5b8560f7ec3b"}} +{"add":{"path":"part-00001-665397f5-1435-4478-a598-ca226c99ffcf-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316825,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":0},\"maxValues\":{\"id\":0},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-fd50becf-074e-4a1f-985b-01529e9f7b03-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316820,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1},\"maxValues\":{\"id\":1},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-4fc1f70f-9daa-46e6-83b5-ea8144d4a96d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316822,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":2},\"maxValues\":{\"id\":2},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-efb0808c-3b7f-4a4d-bc36-daa91c074b5b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316825,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":3},\"maxValues\":{\"id\":3},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-c8664e02-01fe-4c2d-8eba-ae84012d7aad-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316825,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":4},\"maxValues\":{\"id\":4},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-7446ef47-3110-4c3f-a2d0-0c71bafc893a-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316820,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":5},\"maxValues\":{\"id\":5},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-ee372cff-6aae-4979-970b-88cc154a31bd-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316822,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":6},\"maxValues\":{\"id\":6},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-9ea59fdf-fc26-4650-a282-9c2cc1906c7c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316823,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":7},\"maxValues\":{\"id\":7},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-d50ebfbf-e534-4bc8-b63d-437f6029da6e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316823,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":8},\"maxValues\":{\"id\":8},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-96e67376-3d15-4895-bd5d-5e0a325bcb83-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316823,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":9},\"maxValues\":{\"id\":9},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000002.json b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000002.json new file mode 100644 index 0000000000..86e3c7470c --- /dev/null +++ b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000002.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317326453,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":1,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"ac59851b-981e-4e82-96ea-36a543cfe254"}} +{"add":{"path":"part-00001-6491d41d-d498-4a89-a291-92d964035606-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":10},\"maxValues\":{\"id\":10},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-d359921a-3cb1-454d-8aa0-ac5c830fcdc5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":11},\"maxValues\":{\"id\":11},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-b2aae64d-1fab-4106-bc87-2454e945dada-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":12},\"maxValues\":{\"id\":12},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-cca989ea-d56e-4e1e-a4ba-538ef7801997-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":13},\"maxValues\":{\"id\":13},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-14706643-f3f0-4ba9-8282-7d55bb4ecacb-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":14},\"maxValues\":{\"id\":14},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-25801ed5-1cf2-43fa-bbdb-8898fc102e64-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":15},\"maxValues\":{\"id\":15},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-03fbeb6f-b8a5-448e-afa7-0f49fca61866-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":16},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-08361c76-870e-4ddf-9153-f67852849ec3-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":17},\"maxValues\":{\"id\":17},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-368e738e-0673-4e76-a1ff-5ba9c755396e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":18},\"maxValues\":{\"id\":18},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-2c9a3837-e2c5-42bd-b888-f3205f4b894c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":19},\"maxValues\":{\"id\":19},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000003.json b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000003.json new file mode 100644 index 0000000000..14665bcd27 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000003.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317330682,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":2,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"5e6fdfee-2a82-40d1-bc83-47b3f0f3f21e"}} +{"add":{"path":"part-00001-b7be8377-b715-4234-b316-201fd2c9c142-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":20},\"maxValues\":{\"id\":20},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-719b1086-b19c-45d1-8c4c-c11db02e2e0b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":21},\"maxValues\":{\"id\":21},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-eb0fd03c-3e47-42ed-9897-e79dd1567fb1-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":22},\"maxValues\":{\"id\":22},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-48b99dee-6d3e-4cfb-b651-4769de7f5b24-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":23},\"maxValues\":{\"id\":23},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-b873a231-8352-4bac-b6f1-b53ee738d212-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330582,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":24},\"maxValues\":{\"id\":24},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-74ed051c-b116-4947-b62c-2086bcd5bb90-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":25},\"maxValues\":{\"id\":25},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-680d9e49-afce-4a7d-bca8-b03438c2fd74-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":26},\"maxValues\":{\"id\":26},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-81c10052-aacc-4ecf-b9cf-64f81b3bd435-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":27},\"maxValues\":{\"id\":27},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-cf9cac69-932c-43bf-8e4b-fd059d519c0f-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":28},\"maxValues\":{\"id\":28},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-43e3252c-8ac7-4c7a-bcb4-15aaf7ae95b9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":29},\"maxValues\":{\"id\":29},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000004.json b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000004.json new file mode 100644 index 0000000000..ed86e283d5 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000004.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317333588,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":3,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"57611e9e-5312-4502-a3ad-c0c78799773e"}} +{"add":{"path":"part-00001-5c92b4bb-af84-4066-8aeb-1e493b7147df-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333524,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":30},\"maxValues\":{\"id\":30},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-b050a084-ab25-420f-bb7b-50eb95d25e4e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333551,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":31},\"maxValues\":{\"id\":31},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-1880b504-fcf5-4f43-92d4-c43e8dd9d7d9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333550,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":32},\"maxValues\":{\"id\":32},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-4af9173b-8472-41cd-8772-e2bdb084c5d5-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317333524,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":33},\"maxValues\":{\"id\":33},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-89cec234-f844-4802-a786-5d9133bbe489-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333523,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":34},\"maxValues\":{\"id\":34},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-fa11b282-ec0d-4513-9baf-2b84c5f94a12-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333550,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":35},\"maxValues\":{\"id\":35},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-895723cb-0dba-4019-a2a9-e6db9a937c91-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333550,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":36},\"maxValues\":{\"id\":36},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-dca57e7a-f859-4b39-bc43-03e1061f1b4e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333551,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":37},\"maxValues\":{\"id\":37},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-96756753-7714-4c07-a238-d5b57f42a8ce-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333523,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":38},\"maxValues\":{\"id\":38},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-9a7b861f-5d9a-41c4-b4ec-7f0d1391acfe-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333551,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":39},\"maxValues\":{\"id\":39},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000005.json b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000005.json new file mode 100644 index 0000000000..f77fd655fe --- /dev/null +++ b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000005.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317336099,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":4,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"203edd74-d2b2-46fe-935a-6222cfb888d4"}} +{"add":{"path":"part-00001-a24fe71c-ba35-47bb-8f3e-636d5991d5ae-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336058,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":40},\"maxValues\":{\"id\":40},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-a56ad50f-bc64-44cb-bb55-e2d177947b3d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336079,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":41},\"maxValues\":{\"id\":41},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-95c53cf1-b472-4c34-b728-1dd7cbed8b2f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336074,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":42},\"maxValues\":{\"id\":42},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-7e442ced-e810-44d9-9d28-3027e652a0ec-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336080,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":43},\"maxValues\":{\"id\":43},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-68248457-2fa3-407e-9de3-759b1e052b99-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336075,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":44},\"maxValues\":{\"id\":44},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-89a48ead-5bf3-4d16-aada-97c11386fcaf-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336076,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":45},\"maxValues\":{\"id\":45},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-42b618de-c46c-4888-9b48-b99493ec2983-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336070,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":46},\"maxValues\":{\"id\":46},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-4dc49305-f4f8-4ec9-9a40-8f4b3bd81324-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336055,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":47},\"maxValues\":{\"id\":47},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-d7bb17a9-223e-474b-9d78-2c745cc35a4b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336054,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":48},\"maxValues\":{\"id\":48},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-68a79bb6-a31e-49bf-848f-2d64ceb834c0-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336079,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":49},\"maxValues\":{\"id\":49},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000006.json b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000006.json new file mode 100644 index 0000000000..28116ba9f2 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000006.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317338700,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":5,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"9ee4b86b-1bbe-4d6e-adbd-8dd4961989fb"}} +{"add":{"path":"part-00001-a8fc5b00-29e4-4a99-961d-b0cbcc23d165-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338678,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":50},\"maxValues\":{\"id\":50},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-5a3079f8-abbc-4b5f-a1e3-340830e59222-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317338670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":51},\"maxValues\":{\"id\":51},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-567d7b20-b6ce-4e96-b500-caa34c80f8a7-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338676,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":52},\"maxValues\":{\"id\":52},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-b794df4b-174f-468a-9de7-2aa865ba7014-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338676,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":53},\"maxValues\":{\"id\":53},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-a15406cf-c141-4f7b-b302-e4b5a145cad5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338675,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":54},\"maxValues\":{\"id\":54},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-8a52349c-d93b-4c59-b493-13486bb5e284-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338680,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":55},\"maxValues\":{\"id\":55},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-1734b4eb-4414-4b3a-8e99-1bd099c9e6b5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":56},\"maxValues\":{\"id\":56},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-02781f2a-6c34-42ca-80a4-e830b2eeb963-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338676,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":57},\"maxValues\":{\"id\":57},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-63ec1c21-c31c-43d4-b5c9-9c206aeeb280-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":58},\"maxValues\":{\"id\":58},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-b732e8e4-7d1b-470d-89a5-86a3f8d8bdc2-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338680,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":59},\"maxValues\":{\"id\":59},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000007.json b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000007.json new file mode 100644 index 0000000000..956c8b508c --- /dev/null +++ b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000007.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317339658,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":6,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"63c885b0-74eb-4075-a02a-a43b8202b3f8"}} +{"add":{"path":"part-00001-f697bc51-b77f-4234-938a-5f85478cedec-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":60},\"maxValues\":{\"id\":60},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-839f210e-cf84-4c5c-b185-fd2fe2b5ee6f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339491,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":61},\"maxValues\":{\"id\":61},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-9ebe1c22-87a1-4f37-a695-77658c3e70a8-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339491,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":62},\"maxValues\":{\"id\":62},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-01b2687c-45e4-484c-b1d6-80e06b5b5d11-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339490,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":63},\"maxValues\":{\"id\":63},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-05662cc4-6a79-4204-aec1-2311a44d8c74-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339511,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":64},\"maxValues\":{\"id\":64},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-67f1b806-ef5f-4f8a-890b-b3b5ad1d234c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339490,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":65},\"maxValues\":{\"id\":65},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-37318455-4128-4e1e-9ab7-5c587ac9fde0-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":66},\"maxValues\":{\"id\":66},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-c5d68934-1f5a-40c4-b5be-1233eb15378a-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339511,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":67},\"maxValues\":{\"id\":67},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-41e40903-13b6-4465-aa3c-bd8cb5e52b18-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":68},\"maxValues\":{\"id\":68},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-110c626e-ea13-4204-8cae-a3183d89a4b7-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":69},\"maxValues\":{\"id\":69},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000008.json b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000008.json new file mode 100644 index 0000000000..ea216c1556 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000008.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317340794,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":7,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"b5902ae9-2aab-46f9-82f9-d68ae45940d7"}} +{"add":{"path":"part-00001-3b62f1d4-2a3e-4611-a55b-e9d2ace11b3c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340752,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":70},\"maxValues\":{\"id\":70},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-9c066923-23f4-45f6-b2af-5a4ecbef1707-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340776,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":71},\"maxValues\":{\"id\":71},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-cfb48b6e-0fc7-4d6b-8ab7-c52f29f71b94-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340752,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":72},\"maxValues\":{\"id\":72},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-b21a45a5-9a53-4dfa-8327-8a82b6b283e9-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317340766,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":73},\"maxValues\":{\"id\":73},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-afa450c4-f649-4c88-817a-6d0bdfc4da6f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340765,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":74},\"maxValues\":{\"id\":74},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-dd96cd25-394d-4873-84e7-f2f6b0eb5a67-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340776,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":75},\"maxValues\":{\"id\":75},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-3a98b7ed-8665-4bc5-8704-6745f7084cd0-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340776,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":76},\"maxValues\":{\"id\":76},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-8dfd3dc5-cf31-42fc-8c55-2ac70ce9e18d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340782,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":77},\"maxValues\":{\"id\":77},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-97e7d287-aee3-445d-a90e-f3b2ef4bd7cd-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340765,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":78},\"maxValues\":{\"id\":78},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-db2340aa-28ff-4826-b39e-07ba516551e9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340779,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":79},\"maxValues\":{\"id\":79},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000009.json b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000009.json new file mode 100644 index 0000000000..1f9d5fb05b --- /dev/null +++ b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000009.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317341714,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":8,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"5379512e-d4d3-42b5-817d-70ecf05f2385"}} +{"add":{"path":"part-00001-4e7175fd-6ffb-4b6a-946c-43aa7c439104-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":80},\"maxValues\":{\"id\":80},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-f3cc4f07-93ec-4a47-add1-b16c1149c3d9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341667,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":81},\"maxValues\":{\"id\":81},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-07581f2d-ee98-4464-a28b-f738e88749e4-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":82},\"maxValues\":{\"id\":82},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-c9d7468d-fc14-445c-8dbd-65d616f8eb05-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341666,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":83},\"maxValues\":{\"id\":83},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-56eb2260-fb49-4138-a5c8-f0ae0949f4e2-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":84},\"maxValues\":{\"id\":84},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-5d15eeca-0fca-4986-a18e-4d86bf5ba2f6-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":85},\"maxValues\":{\"id\":85},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-3e13d733-f55d-42ac-be4f-f4400e999c29-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":86},\"maxValues\":{\"id\":86},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-22c23f1e-26d2-488e-8e07-2de6ae5fded5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341667,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":87},\"maxValues\":{\"id\":87},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-d0debf09-5f82-4c61-8636-27e51fba37e5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341674,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":88},\"maxValues\":{\"id\":88},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-7403e0c4-bb07-4a1b-9fca-a01523713f85-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":89},\"maxValues\":{\"id\":89},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000010.checkpoint.parquet b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000010.checkpoint.parquet new file mode 100644 index 0000000000..9df253f2f7 Binary files /dev/null and b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000010.checkpoint.parquet differ diff --git a/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000010.json b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000010.json new file mode 100644 index 0000000000..c1f8c8b095 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000010.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317342605,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":9,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"ba4b200d-e0ac-4715-ad4e-bed8ef1b20cd"}} +{"add":{"path":"part-00001-55fbea2e-0788-438a-a50c-65f809acc05b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":90},\"maxValues\":{\"id\":90},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-0d891eaf-0e01-46a6-879e-49bbca90c215-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":91},\"maxValues\":{\"id\":91},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-569d12e9-04d5-4fe2-9554-f288047f3386-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342592,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":92},\"maxValues\":{\"id\":92},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-1bc89ec9-8092-49e1-9b1f-123ae50e3d40-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342579,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":93},\"maxValues\":{\"id\":93},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-b4a223d6-1d87-49c9-84c9-a85eece61839-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342591,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":94},\"maxValues\":{\"id\":94},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-4c3b6be7-979c-4f42-8920-efa32b751d97-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":95},\"maxValues\":{\"id\":95},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-6c224359-8995-417a-8b24-b2e530327bc6-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":96},\"maxValues\":{\"id\":96},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-2a40eb21-34d2-48ca-aaa5-55db674f56de-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342590,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":97},\"maxValues\":{\"id\":97},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-ca8256ed-98cd-460d-8de2-9f6f7f388703-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342579,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":98},\"maxValues\":{\"id\":98},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-f131fc78-c201-4e8d-b194-222b2e79778d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":99},\"maxValues\":{\"id\":99},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000011.json b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000011.json new file mode 100644 index 0000000000..05863f6cd0 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000011.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317349152,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":10,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"1657fc30-879d-4b0b-972a-4e3a079fdd7a"}} +{"add":{"path":"part-00001-ceaadd5e-615b-455d-8f4b-052b9c94c7b6-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349136,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":100},\"maxValues\":{\"id\":100},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-b7dba1e7-b1e5-4f02-a223-69ec7353ab45-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349123,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":101},\"maxValues\":{\"id\":101},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-f099cf4d-d418-4852-8580-091908847a66-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317349122,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":102},\"maxValues\":{\"id\":102},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-418e8d25-7316-442a-9bc8-616ed01231eb-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349136,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":103},\"maxValues\":{\"id\":103},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-7538a9c2-1ccb-4150-b162-ef8d826fe30f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349123,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":104},\"maxValues\":{\"id\":104},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-7619f42c-5bc4-4e77-b037-f36481c8b63c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349123,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":105},\"maxValues\":{\"id\":105},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-eb49d85f-91cc-4293-9339-a664ee905b0f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349134,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":106},\"maxValues\":{\"id\":106},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-81e22719-7705-4703-b2dd-c4e2982217a7-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349122,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":107},\"maxValues\":{\"id\":107},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-883858d1-9df6-4b55-a2be-5b8387134617-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349122,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":108},\"maxValues\":{\"id\":108},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-07d91938-ac89-48cc-a657-6067d2d9f67e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349137,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":109},\"maxValues\":{\"id\":109},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000013.json b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000013.json new file mode 100644 index 0000000000..b2d03d3ead --- /dev/null +++ b/crates/core/tests/data_err_logs/table_c/_delta_log/00000000000000000013.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317350712,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":12,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"c3cd0fa1-9c72-4344-8225-0b787e52d5e0"}} +{"add":{"path":"part-00001-7a0d95f8-e122-4cf6-b89c-389036a9b415-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350696,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":120},\"maxValues\":{\"id\":120},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-f1f035c1-bf0f-485c-950d-c81d0d2aa8a2-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350706,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":121},\"maxValues\":{\"id\":121},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-d7a51e45-70f3-4379-819b-341951abefff-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350691,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":122},\"maxValues\":{\"id\":122},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-4828722c-5799-4be1-ace1-14bd7f477dbf-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350691,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":123},\"maxValues\":{\"id\":123},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-f3c3c72e-5d71-4dc9-9e15-342f1d6cb6cc-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350701,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":124},\"maxValues\":{\"id\":124},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-e8d74ede-8876-4f55-8e9f-1bbde0d07a35-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350696,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":125},\"maxValues\":{\"id\":125},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-91052146-2292-45c3-b57e-1fd2dd6be6ed-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350692,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":126},\"maxValues\":{\"id\":126},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-9d203964-9f4c-4c84-ad77-9ba305bb6572-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350706,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":127},\"maxValues\":{\"id\":127},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-8362228b-acf6-4937-875b-26c013c342e1-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350690,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":128},\"maxValues\":{\"id\":128},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-6f57658e-1953-4b59-b504-27c9e8c5cc3b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350677,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":129},\"maxValues\":{\"id\":129},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_c/_delta_log/_last_checkpoint b/crates/core/tests/data_err_logs/table_c/_delta_log/_last_checkpoint new file mode 100644 index 0000000000..604c2b31a2 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_c/_delta_log/_last_checkpoint @@ -0,0 +1 @@ +{"version":10,"size":102,"sizeInBytes":21615,"numOfAddFiles":100,"checkpointSchema":{"type":"struct","fields":[{"name":"txn","type":{"type":"struct","fields":[{"name":"appId","type":"string","nullable":true,"metadata":{}},{"name":"version","type":"long","nullable":true,"metadata":{}},{"name":"lastUpdated","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"add","type":{"type":"struct","fields":[{"name":"path","type":"string","nullable":true,"metadata":{}},{"name":"partitionValues","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"size","type":"long","nullable":true,"metadata":{}},{"name":"modificationTime","type":"long","nullable":true,"metadata":{}},{"name":"dataChange","type":"boolean","nullable":true,"metadata":{}},{"name":"tags","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"deletionVector","type":{"type":"struct","fields":[{"name":"storageType","type":"string","nullable":true,"metadata":{}},{"name":"pathOrInlineDv","type":"string","nullable":true,"metadata":{}},{"name":"offset","type":"integer","nullable":true,"metadata":{}},{"name":"sizeInBytes","type":"integer","nullable":true,"metadata":{}},{"name":"cardinality","type":"long","nullable":true,"metadata":{}},{"name":"maxRowIndex","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"baseRowId","type":"long","nullable":true,"metadata":{}},{"name":"defaultRowCommitVersion","type":"long","nullable":true,"metadata":{}},{"name":"clusteringProvider","type":"string","nullable":true,"metadata":{}},{"name":"stats","type":"string","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"remove","type":{"type":"struct","fields":[{"name":"path","type":"string","nullable":true,"metadata":{}},{"name":"deletionTimestamp","type":"long","nullable":true,"metadata":{}},{"name":"dataChange","type":"boolean","nullable":true,"metadata":{}},{"name":"extendedFileMetadata","type":"boolean","nullable":true,"metadata":{}},{"name":"partitionValues","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"size","type":"long","nullable":true,"metadata":{}},{"name":"deletionVector","type":{"type":"struct","fields":[{"name":"storageType","type":"string","nullable":true,"metadata":{}},{"name":"pathOrInlineDv","type":"string","nullable":true,"metadata":{}},{"name":"offset","type":"integer","nullable":true,"metadata":{}},{"name":"sizeInBytes","type":"integer","nullable":true,"metadata":{}},{"name":"cardinality","type":"long","nullable":true,"metadata":{}},{"name":"maxRowIndex","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"baseRowId","type":"long","nullable":true,"metadata":{}},{"name":"defaultRowCommitVersion","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"metaData","type":{"type":"struct","fields":[{"name":"id","type":"string","nullable":true,"metadata":{}},{"name":"name","type":"string","nullable":true,"metadata":{}},{"name":"description","type":"string","nullable":true,"metadata":{}},{"name":"format","type":{"type":"struct","fields":[{"name":"provider","type":"string","nullable":true,"metadata":{}},{"name":"options","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"schemaString","type":"string","nullable":true,"metadata":{}},{"name":"partitionColumns","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"configuration","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"createdTime","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"protocol","type":{"type":"struct","fields":[{"name":"minReaderVersion","type":"integer","nullable":true,"metadata":{}},{"name":"minWriterVersion","type":"integer","nullable":true,"metadata":{}},{"name":"readerFeatures","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"writerFeatures","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"domainMetadata","type":{"type":"struct","fields":[{"name":"domain","type":"string","nullable":true,"metadata":{}},{"name":"configuration","type":"string","nullable":true,"metadata":{}},{"name":"removed","type":"boolean","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}}]},"checksum":"94a578f92841fa7ba9cdee96b5905fdb"} diff --git a/crates/core/tests/data_err_logs/table_d/_delta_log/00000000000000000000.checkpoint.parquet b/crates/core/tests/data_err_logs/table_d/_delta_log/00000000000000000000.checkpoint.parquet new file mode 100644 index 0000000000..9fd1755fd5 Binary files /dev/null and b/crates/core/tests/data_err_logs/table_d/_delta_log/00000000000000000000.checkpoint.parquet differ diff --git a/crates/core/tests/data_err_logs/table_d/_delta_log/00000000000000000000.json b/crates/core/tests/data_err_logs/table_d/_delta_log/00000000000000000000.json new file mode 100644 index 0000000000..d78fe64124 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_d/_delta_log/00000000000000000000.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1742252106605,"engineInfo":"Kernel-3.4.0-SNAPSHOT/test-engine","operation":"CREATE TABLE","operationParameters":{"partitionBy":"[]"},"isBlindAppend":true,"txnId":"138db460-c7d3-4d13-972e-0f4e35b58b43","operationMetrics":{}}} +{"metaData":{"id":"ebffcf7b-bf25-427f-a91a-d4a0d27ebbef","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"createdTime":1742252106422,"configuration":{"delta.feature.catalogowned":"supported"}}} +{"protocol":{"minReaderVersion":3,"minWriterVersion":7,"readerFeatures":["catalogOwned"],"writerFeatures":["catalogOwned","invariants","appendOnly"]}} diff --git a/crates/core/tests/data_err_logs/table_d/_delta_log/_last_checkpoint b/crates/core/tests/data_err_logs/table_d/_delta_log/_last_checkpoint new file mode 100644 index 0000000000..f55c81779f --- /dev/null +++ b/crates/core/tests/data_err_logs/table_d/_delta_log/_last_checkpoint @@ -0,0 +1 @@ +{"version":0,"size":0} diff --git a/crates/core/tests/data_err_logs/table_e/_delta_log/00000000000000000000.checkpoint.parquet b/crates/core/tests/data_err_logs/table_e/_delta_log/00000000000000000000.checkpoint.parquet new file mode 100644 index 0000000000..4d9437c57f Binary files /dev/null and b/crates/core/tests/data_err_logs/table_e/_delta_log/00000000000000000000.checkpoint.parquet differ diff --git a/crates/core/tests/data_err_logs/table_e/_delta_log/00000000000000000000.json b/crates/core/tests/data_err_logs/table_e/_delta_log/00000000000000000000.json new file mode 100644 index 0000000000..37d128c02c --- /dev/null +++ b/crates/core/tests/data_err_logs/table_e/_delta_log/00000000000000000000.json @@ -0,0 +1,4 @@ +{"commitInfo":{"timestamp":1742317197774,"engineInfo":"Kernel-3.4.0-SNAPSHOT/test-engine","operation":"CREATE TABLE","operationParameters":{"partitionBy":"[]"},"isBlindAppend":true,"txnId":"09e63766-07ed-4b57-a6e0-0bba30aa801e","operationMetrics":{}}} +{"metaData":{"id":"7998c165-cb06-408d-bba0-e5c4e532ba21","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"createdTime":1742317196814,"configuration":{"delta.feature.catalogowned":"supported"}}} +{"protocol":{"minReaderVersion":3,"minWriterVersion":7,"readerFeatures":["catalogOwned"],"writerFeatures":["catalogOwned","appendOnly","invariants"]}} +{"add":{"path":"a735be79-daea-40b6-94f0-317a77a03df6-000.parquet","partitionValues":{},"size":1918,"modificationTime":1742317197691,"dataChange":true,"stats":"{\"numRecords\":600,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":3251},\"nullCounts\":{\"id\":30}}"}} diff --git a/crates/core/tests/data_err_logs/table_e/_delta_log/_last_checkpoint b/crates/core/tests/data_err_logs/table_e/_delta_log/_last_checkpoint new file mode 100644 index 0000000000..74f27fded1 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_e/_delta_log/_last_checkpoint @@ -0,0 +1 @@ +{"version":0,"size":1} diff --git a/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000000.json b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000000.json new file mode 100644 index 0000000000..25bdb69d88 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000000.json @@ -0,0 +1,4 @@ +{"commitInfo":{"timestamp":1742317796445,"engineInfo":"Kernel-3.4.0-SNAPSHOT/test-engine","operation":"CREATE TABLE","operationParameters":{"partitionBy":"[]"},"isBlindAppend":true,"txnId":"3d3a3463-1ec9-428d-bdc0-8dd8bad69aee","operationMetrics":{}}} +{"metaData":{"id":"f8d0b1b0-efb8-4323-9c35-69d18d2eebc9","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"createdTime":1742317795464,"configuration":{"delta.feature.catalogowned":"supported"}}} +{"protocol":{"minReaderVersion":3,"minWriterVersion":7,"readerFeatures":["catalogOwned"],"writerFeatures":["catalogOwned","invariants","appendOnly"]}} +{"add":{"path":"67cfa9cb-6933-4e2c-90b3-5b0c5b513e35-000.parquet","partitionValues":{},"size":1918,"modificationTime":1742317796362,"dataChange":true,"stats":"{\"numRecords\":600,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":3251},\"nullCounts\":{\"id\":30}}"}} diff --git a/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000001.json b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000001.json new file mode 100644 index 0000000000..3d66057360 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000001.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1742317797041,"engineInfo":"Kernel-3.4.0-SNAPSHOT/test-engine","operation":"WRITE","operationParameters":{},"isBlindAppend":true,"txnId":"7bce2352-93ed-4f0c-9f63-42b8600d93d9","operationMetrics":{}}} +{"add":{"path":"ea72200d-614f-42d0-873e-86e7b9442afa-000.parquet","partitionValues":{},"size":1918,"modificationTime":1742317797038,"dataChange":true,"stats":"{\"numRecords\":600,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":3251},\"nullCounts\":{\"id\":30}}"}} diff --git a/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000002.json b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000002.json new file mode 100644 index 0000000000..e4603c5c6b --- /dev/null +++ b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000002.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1742317797595,"engineInfo":"Kernel-3.4.0-SNAPSHOT/test-engine","operation":"WRITE","operationParameters":{},"isBlindAppend":true,"txnId":"14f8e260-767a-498e-bcfb-3b18eda1edd5","operationMetrics":{}}} +{"add":{"path":"b86a2fe9-f9d5-415b-ae8e-78b9c46ac18c-000.parquet","partitionValues":{},"size":1918,"modificationTime":1742317797587,"dataChange":true,"stats":"{\"numRecords\":600,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":3251},\"nullCounts\":{\"id\":30}}"}} diff --git a/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000003.json b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000003.json new file mode 100644 index 0000000000..43abaf0510 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000003.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1742317798121,"engineInfo":"Kernel-3.4.0-SNAPSHOT/test-engine","operation":"WRITE","operationParameters":{},"isBlindAppend":true,"txnId":"f802c7fd-4be1-43e2-a7bc-33a1737ecb2d","operationMetrics":{}}} +{"add":{"path":"1971092a-aabc-4a36-a792-a4ec7f98084a-000.parquet","partitionValues":{},"size":1918,"modificationTime":1742317798117,"dataChange":true,"stats":"{\"numRecords\":600,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":3251},\"nullCounts\":{\"id\":30}}"}} diff --git a/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000004.json b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000004.json new file mode 100644 index 0000000000..06abe1b599 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000004.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1742317798598,"engineInfo":"Kernel-3.4.0-SNAPSHOT/test-engine","operation":"WRITE","operationParameters":{},"isBlindAppend":true,"txnId":"d460c3db-4f07-43dc-9c93-1f43dce18f41","operationMetrics":{}}} +{"add":{"path":"e18d2d0e-b059-4842-8e9a-773ce0289a71-000.parquet","partitionValues":{},"size":1918,"modificationTime":1742317798593,"dataChange":true,"stats":"{\"numRecords\":600,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":3251},\"nullCounts\":{\"id\":30}}"}} diff --git a/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000005.json b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000005.json new file mode 100644 index 0000000000..05956e13f1 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000005.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1742317799048,"engineInfo":"Kernel-3.4.0-SNAPSHOT/test-engine","operation":"WRITE","operationParameters":{},"isBlindAppend":true,"txnId":"da5d8fc4-b50b-4287-9362-080ed61211b2","operationMetrics":{}}} +{"add":{"path":"a1885a41-cf1f-43b2-9795-013182d51033-000.parquet","partitionValues":{},"size":1918,"modificationTime":1742317799039,"dataChange":true,"stats":"{\"numRecords\":600,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":3251},\"nullCounts\":{\"id\":30}}"}} diff --git a/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000006.json b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000006.json new file mode 100644 index 0000000000..065299eb12 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000006.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1742317799455,"engineInfo":"Kernel-3.4.0-SNAPSHOT/test-engine","operation":"WRITE","operationParameters":{},"isBlindAppend":true,"txnId":"a415edd2-9413-4635-b864-e859053e6902","operationMetrics":{}}} +{"add":{"path":"05ce144c-628e-463c-8a49-b9a3e2a0f03a-000.parquet","partitionValues":{},"size":1918,"modificationTime":1742317799453,"dataChange":true,"stats":"{\"numRecords\":600,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":3251},\"nullCounts\":{\"id\":30}}"}} diff --git a/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000007.json b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000007.json new file mode 100644 index 0000000000..6353238177 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000007.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1742317799859,"engineInfo":"Kernel-3.4.0-SNAPSHOT/test-engine","operation":"WRITE","operationParameters":{},"isBlindAppend":true,"txnId":"67e8727d-3d94-4e38-8c42-f86e21912a5c","operationMetrics":{}}} +{"add":{"path":"94443ade-9c5d-4efc-9bd6-fd5d39bd3925-000.parquet","partitionValues":{},"size":1918,"modificationTime":1742317799854,"dataChange":true,"stats":"{\"numRecords\":600,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":3251},\"nullCounts\":{\"id\":30}}"}} diff --git a/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000008.json b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000008.json new file mode 100644 index 0000000000..f891a3f55d --- /dev/null +++ b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000008.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1742317800272,"engineInfo":"Kernel-3.4.0-SNAPSHOT/test-engine","operation":"WRITE","operationParameters":{},"isBlindAppend":true,"txnId":"83cbc057-f514-4cad-a563-47894d21a89a","operationMetrics":{}}} +{"add":{"path":"12a92422-f105-4d85-af99-248e16bdaaa1-000.parquet","partitionValues":{},"size":1918,"modificationTime":1742317800269,"dataChange":true,"stats":"{\"numRecords\":600,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":3251},\"nullCounts\":{\"id\":30}}"}} diff --git a/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000009.json b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000009.json new file mode 100644 index 0000000000..0b9cd01640 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000009.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1742317800696,"engineInfo":"Kernel-3.4.0-SNAPSHOT/test-engine","operation":"WRITE","operationParameters":{},"isBlindAppend":true,"txnId":"ef305401-d8f7-4db7-afff-31ae85d6b78c","operationMetrics":{}}} +{"add":{"path":"a236c3b0-99e6-4ae6-aa49-2f90b1e3cb2c-000.parquet","partitionValues":{},"size":1918,"modificationTime":1742317800681,"dataChange":true,"stats":"{\"numRecords\":600,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":3251},\"nullCounts\":{\"id\":30}}"}} diff --git a/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000010.checkpoint.parquet b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000010.checkpoint.parquet new file mode 100644 index 0000000000..9d4d415f06 Binary files /dev/null and b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000010.checkpoint.parquet differ diff --git a/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000010.json b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000010.json new file mode 100644 index 0000000000..7c88a20348 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000010.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1742317801123,"engineInfo":"Kernel-3.4.0-SNAPSHOT/test-engine","operation":"WRITE","operationParameters":{},"isBlindAppend":true,"txnId":"9e00a4c2-bc6f-43d3-95cf-19fcf5be5015","operationMetrics":{}}} +{"add":{"path":"2b2f2b8c-fa9b-4f5d-972a-d49dc2576525-000.parquet","partitionValues":{},"size":1918,"modificationTime":1742317801121,"dataChange":true,"stats":"{\"numRecords\":600,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":3251},\"nullCounts\":{\"id\":30}}"}} diff --git a/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000011.json b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000011.json new file mode 100644 index 0000000000..99f46d5427 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000011.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1742317801512,"engineInfo":"Kernel-3.4.0-SNAPSHOT/test-engine","operation":"WRITE","operationParameters":{},"isBlindAppend":true,"txnId":"0f8b8815-8ef0-48f4-a936-709a8ea9eb01","operationMetrics":{}}} +{"add":{"path":"77f386e1-5720-441c-98f1-6669caee1436-000.parquet","partitionValues":{},"size":1918,"modificationTime":1742317801511,"dataChange":true,"stats":"{\"numRecords\":600,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":3251},\"nullCounts\":{\"id\":30}}"}} diff --git a/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000012.json b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000012.json new file mode 100644 index 0000000000..b01cf903be --- /dev/null +++ b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000012.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1742317801905,"engineInfo":"Kernel-3.4.0-SNAPSHOT/test-engine","operation":"WRITE","operationParameters":{},"isBlindAppend":true,"txnId":"6b95204e-13c7-4de9-be37-43295f4cec65","operationMetrics":{}}} +{"add":{"path":"f1f34c16-7bc6-4033-80f2-499fe408439d-000.parquet","partitionValues":{},"size":1918,"modificationTime":1742317801902,"dataChange":true,"stats":"{\"numRecords\":600,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":3251},\"nullCounts\":{\"id\":30}}"}} diff --git a/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000013.json b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000013.json new file mode 100644 index 0000000000..24666b9e6d --- /dev/null +++ b/crates/core/tests/data_err_logs/table_f/_delta_log/00000000000000000013.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1742317802323,"engineInfo":"Kernel-3.4.0-SNAPSHOT/test-engine","operation":"WRITE","operationParameters":{},"isBlindAppend":true,"txnId":"c3e5a1a9-e706-40f8-81e0-ec082b859523","operationMetrics":{}}} +{"add":{"path":"9df4a1e6-9198-4888-ba7c-8e3f97ec8981-000.parquet","partitionValues":{},"size":1918,"modificationTime":1742317802318,"dataChange":true,"stats":"{\"numRecords\":600,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":3251},\"nullCounts\":{\"id\":30}}"}} diff --git a/crates/core/tests/data_err_logs/table_f/_delta_log/_last_checkpoint b/crates/core/tests/data_err_logs/table_f/_delta_log/_last_checkpoint new file mode 100644 index 0000000000..56688d647c --- /dev/null +++ b/crates/core/tests/data_err_logs/table_f/_delta_log/_last_checkpoint @@ -0,0 +1 @@ +{"version":10,"size":11} diff --git a/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000000.json b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000000.json new file mode 100644 index 0000000000..0316f09771 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000000.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1742317262289,"operation":"CREATE TABLE","operationParameters":{"partitionBy":"[]","clusterBy":"[]","description":null,"isManaged":"false","properties":"{}"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"c0983a7a-aeca-4ba8-a509-d5dbc71a10de"}} +{"metaData":{"id":"5f54ef5f-e511-4114-b6e5-f6c206c068b6","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1742317261939}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":1}} diff --git a/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000001.json b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000001.json new file mode 100644 index 0000000000..23002c6f58 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000001.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317316973,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"a1c49e36-8fcd-46e6-8ede-5b8560f7ec3b"}} +{"add":{"path":"part-00001-665397f5-1435-4478-a598-ca226c99ffcf-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316825,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":0},\"maxValues\":{\"id\":0},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-fd50becf-074e-4a1f-985b-01529e9f7b03-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316820,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1},\"maxValues\":{\"id\":1},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-4fc1f70f-9daa-46e6-83b5-ea8144d4a96d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316822,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":2},\"maxValues\":{\"id\":2},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-efb0808c-3b7f-4a4d-bc36-daa91c074b5b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316825,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":3},\"maxValues\":{\"id\":3},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-c8664e02-01fe-4c2d-8eba-ae84012d7aad-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316825,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":4},\"maxValues\":{\"id\":4},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-7446ef47-3110-4c3f-a2d0-0c71bafc893a-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316820,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":5},\"maxValues\":{\"id\":5},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-ee372cff-6aae-4979-970b-88cc154a31bd-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316822,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":6},\"maxValues\":{\"id\":6},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-9ea59fdf-fc26-4650-a282-9c2cc1906c7c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316823,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":7},\"maxValues\":{\"id\":7},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-d50ebfbf-e534-4bc8-b63d-437f6029da6e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316823,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":8},\"maxValues\":{\"id\":8},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-96e67376-3d15-4895-bd5d-5e0a325bcb83-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316823,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":9},\"maxValues\":{\"id\":9},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000002.json b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000002.json new file mode 100644 index 0000000000..86e3c7470c --- /dev/null +++ b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000002.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317326453,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":1,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"ac59851b-981e-4e82-96ea-36a543cfe254"}} +{"add":{"path":"part-00001-6491d41d-d498-4a89-a291-92d964035606-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":10},\"maxValues\":{\"id\":10},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-d359921a-3cb1-454d-8aa0-ac5c830fcdc5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":11},\"maxValues\":{\"id\":11},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-b2aae64d-1fab-4106-bc87-2454e945dada-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":12},\"maxValues\":{\"id\":12},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-cca989ea-d56e-4e1e-a4ba-538ef7801997-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":13},\"maxValues\":{\"id\":13},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-14706643-f3f0-4ba9-8282-7d55bb4ecacb-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":14},\"maxValues\":{\"id\":14},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-25801ed5-1cf2-43fa-bbdb-8898fc102e64-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":15},\"maxValues\":{\"id\":15},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-03fbeb6f-b8a5-448e-afa7-0f49fca61866-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":16},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-08361c76-870e-4ddf-9153-f67852849ec3-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":17},\"maxValues\":{\"id\":17},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-368e738e-0673-4e76-a1ff-5ba9c755396e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":18},\"maxValues\":{\"id\":18},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-2c9a3837-e2c5-42bd-b888-f3205f4b894c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":19},\"maxValues\":{\"id\":19},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000003.json b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000003.json new file mode 100644 index 0000000000..14665bcd27 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000003.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317330682,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":2,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"5e6fdfee-2a82-40d1-bc83-47b3f0f3f21e"}} +{"add":{"path":"part-00001-b7be8377-b715-4234-b316-201fd2c9c142-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":20},\"maxValues\":{\"id\":20},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-719b1086-b19c-45d1-8c4c-c11db02e2e0b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":21},\"maxValues\":{\"id\":21},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-eb0fd03c-3e47-42ed-9897-e79dd1567fb1-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":22},\"maxValues\":{\"id\":22},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-48b99dee-6d3e-4cfb-b651-4769de7f5b24-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":23},\"maxValues\":{\"id\":23},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-b873a231-8352-4bac-b6f1-b53ee738d212-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330582,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":24},\"maxValues\":{\"id\":24},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-74ed051c-b116-4947-b62c-2086bcd5bb90-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":25},\"maxValues\":{\"id\":25},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-680d9e49-afce-4a7d-bca8-b03438c2fd74-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":26},\"maxValues\":{\"id\":26},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-81c10052-aacc-4ecf-b9cf-64f81b3bd435-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":27},\"maxValues\":{\"id\":27},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-cf9cac69-932c-43bf-8e4b-fd059d519c0f-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":28},\"maxValues\":{\"id\":28},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-43e3252c-8ac7-4c7a-bcb4-15aaf7ae95b9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":29},\"maxValues\":{\"id\":29},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000004.json b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000004.json new file mode 100644 index 0000000000..ed86e283d5 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000004.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317333588,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":3,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"57611e9e-5312-4502-a3ad-c0c78799773e"}} +{"add":{"path":"part-00001-5c92b4bb-af84-4066-8aeb-1e493b7147df-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333524,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":30},\"maxValues\":{\"id\":30},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-b050a084-ab25-420f-bb7b-50eb95d25e4e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333551,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":31},\"maxValues\":{\"id\":31},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-1880b504-fcf5-4f43-92d4-c43e8dd9d7d9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333550,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":32},\"maxValues\":{\"id\":32},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-4af9173b-8472-41cd-8772-e2bdb084c5d5-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317333524,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":33},\"maxValues\":{\"id\":33},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-89cec234-f844-4802-a786-5d9133bbe489-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333523,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":34},\"maxValues\":{\"id\":34},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-fa11b282-ec0d-4513-9baf-2b84c5f94a12-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333550,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":35},\"maxValues\":{\"id\":35},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-895723cb-0dba-4019-a2a9-e6db9a937c91-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333550,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":36},\"maxValues\":{\"id\":36},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-dca57e7a-f859-4b39-bc43-03e1061f1b4e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333551,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":37},\"maxValues\":{\"id\":37},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-96756753-7714-4c07-a238-d5b57f42a8ce-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333523,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":38},\"maxValues\":{\"id\":38},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-9a7b861f-5d9a-41c4-b4ec-7f0d1391acfe-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333551,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":39},\"maxValues\":{\"id\":39},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000005.json b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000005.json new file mode 100644 index 0000000000..f77fd655fe --- /dev/null +++ b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000005.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317336099,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":4,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"203edd74-d2b2-46fe-935a-6222cfb888d4"}} +{"add":{"path":"part-00001-a24fe71c-ba35-47bb-8f3e-636d5991d5ae-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336058,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":40},\"maxValues\":{\"id\":40},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-a56ad50f-bc64-44cb-bb55-e2d177947b3d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336079,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":41},\"maxValues\":{\"id\":41},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-95c53cf1-b472-4c34-b728-1dd7cbed8b2f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336074,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":42},\"maxValues\":{\"id\":42},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-7e442ced-e810-44d9-9d28-3027e652a0ec-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336080,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":43},\"maxValues\":{\"id\":43},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-68248457-2fa3-407e-9de3-759b1e052b99-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336075,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":44},\"maxValues\":{\"id\":44},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-89a48ead-5bf3-4d16-aada-97c11386fcaf-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336076,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":45},\"maxValues\":{\"id\":45},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-42b618de-c46c-4888-9b48-b99493ec2983-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336070,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":46},\"maxValues\":{\"id\":46},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-4dc49305-f4f8-4ec9-9a40-8f4b3bd81324-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336055,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":47},\"maxValues\":{\"id\":47},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-d7bb17a9-223e-474b-9d78-2c745cc35a4b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336054,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":48},\"maxValues\":{\"id\":48},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-68a79bb6-a31e-49bf-848f-2d64ceb834c0-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336079,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":49},\"maxValues\":{\"id\":49},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000006.json b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000006.json new file mode 100644 index 0000000000..28116ba9f2 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000006.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317338700,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":5,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"9ee4b86b-1bbe-4d6e-adbd-8dd4961989fb"}} +{"add":{"path":"part-00001-a8fc5b00-29e4-4a99-961d-b0cbcc23d165-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338678,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":50},\"maxValues\":{\"id\":50},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-5a3079f8-abbc-4b5f-a1e3-340830e59222-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317338670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":51},\"maxValues\":{\"id\":51},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-567d7b20-b6ce-4e96-b500-caa34c80f8a7-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338676,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":52},\"maxValues\":{\"id\":52},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-b794df4b-174f-468a-9de7-2aa865ba7014-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338676,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":53},\"maxValues\":{\"id\":53},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-a15406cf-c141-4f7b-b302-e4b5a145cad5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338675,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":54},\"maxValues\":{\"id\":54},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-8a52349c-d93b-4c59-b493-13486bb5e284-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338680,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":55},\"maxValues\":{\"id\":55},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-1734b4eb-4414-4b3a-8e99-1bd099c9e6b5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":56},\"maxValues\":{\"id\":56},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-02781f2a-6c34-42ca-80a4-e830b2eeb963-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338676,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":57},\"maxValues\":{\"id\":57},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-63ec1c21-c31c-43d4-b5c9-9c206aeeb280-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":58},\"maxValues\":{\"id\":58},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-b732e8e4-7d1b-470d-89a5-86a3f8d8bdc2-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338680,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":59},\"maxValues\":{\"id\":59},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000007.json b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000007.json new file mode 100644 index 0000000000..956c8b508c --- /dev/null +++ b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000007.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317339658,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":6,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"63c885b0-74eb-4075-a02a-a43b8202b3f8"}} +{"add":{"path":"part-00001-f697bc51-b77f-4234-938a-5f85478cedec-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":60},\"maxValues\":{\"id\":60},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-839f210e-cf84-4c5c-b185-fd2fe2b5ee6f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339491,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":61},\"maxValues\":{\"id\":61},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-9ebe1c22-87a1-4f37-a695-77658c3e70a8-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339491,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":62},\"maxValues\":{\"id\":62},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-01b2687c-45e4-484c-b1d6-80e06b5b5d11-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339490,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":63},\"maxValues\":{\"id\":63},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-05662cc4-6a79-4204-aec1-2311a44d8c74-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339511,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":64},\"maxValues\":{\"id\":64},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-67f1b806-ef5f-4f8a-890b-b3b5ad1d234c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339490,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":65},\"maxValues\":{\"id\":65},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-37318455-4128-4e1e-9ab7-5c587ac9fde0-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":66},\"maxValues\":{\"id\":66},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-c5d68934-1f5a-40c4-b5be-1233eb15378a-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339511,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":67},\"maxValues\":{\"id\":67},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-41e40903-13b6-4465-aa3c-bd8cb5e52b18-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":68},\"maxValues\":{\"id\":68},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-110c626e-ea13-4204-8cae-a3183d89a4b7-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":69},\"maxValues\":{\"id\":69},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000008.json b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000008.json new file mode 100644 index 0000000000..ea216c1556 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000008.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317340794,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":7,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"b5902ae9-2aab-46f9-82f9-d68ae45940d7"}} +{"add":{"path":"part-00001-3b62f1d4-2a3e-4611-a55b-e9d2ace11b3c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340752,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":70},\"maxValues\":{\"id\":70},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-9c066923-23f4-45f6-b2af-5a4ecbef1707-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340776,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":71},\"maxValues\":{\"id\":71},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-cfb48b6e-0fc7-4d6b-8ab7-c52f29f71b94-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340752,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":72},\"maxValues\":{\"id\":72},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-b21a45a5-9a53-4dfa-8327-8a82b6b283e9-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317340766,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":73},\"maxValues\":{\"id\":73},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-afa450c4-f649-4c88-817a-6d0bdfc4da6f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340765,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":74},\"maxValues\":{\"id\":74},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-dd96cd25-394d-4873-84e7-f2f6b0eb5a67-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340776,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":75},\"maxValues\":{\"id\":75},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-3a98b7ed-8665-4bc5-8704-6745f7084cd0-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340776,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":76},\"maxValues\":{\"id\":76},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-8dfd3dc5-cf31-42fc-8c55-2ac70ce9e18d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340782,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":77},\"maxValues\":{\"id\":77},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-97e7d287-aee3-445d-a90e-f3b2ef4bd7cd-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340765,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":78},\"maxValues\":{\"id\":78},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-db2340aa-28ff-4826-b39e-07ba516551e9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340779,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":79},\"maxValues\":{\"id\":79},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000009.json b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000009.json new file mode 100644 index 0000000000..1f9d5fb05b --- /dev/null +++ b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000009.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317341714,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":8,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"5379512e-d4d3-42b5-817d-70ecf05f2385"}} +{"add":{"path":"part-00001-4e7175fd-6ffb-4b6a-946c-43aa7c439104-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":80},\"maxValues\":{\"id\":80},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-f3cc4f07-93ec-4a47-add1-b16c1149c3d9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341667,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":81},\"maxValues\":{\"id\":81},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-07581f2d-ee98-4464-a28b-f738e88749e4-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":82},\"maxValues\":{\"id\":82},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-c9d7468d-fc14-445c-8dbd-65d616f8eb05-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341666,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":83},\"maxValues\":{\"id\":83},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-56eb2260-fb49-4138-a5c8-f0ae0949f4e2-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":84},\"maxValues\":{\"id\":84},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-5d15eeca-0fca-4986-a18e-4d86bf5ba2f6-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":85},\"maxValues\":{\"id\":85},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-3e13d733-f55d-42ac-be4f-f4400e999c29-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":86},\"maxValues\":{\"id\":86},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-22c23f1e-26d2-488e-8e07-2de6ae5fded5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341667,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":87},\"maxValues\":{\"id\":87},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-d0debf09-5f82-4c61-8636-27e51fba37e5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341674,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":88},\"maxValues\":{\"id\":88},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-7403e0c4-bb07-4a1b-9fca-a01523713f85-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":89},\"maxValues\":{\"id\":89},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000010.checkpoint.parquet b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000010.checkpoint.parquet new file mode 100644 index 0000000000..9df253f2f7 Binary files /dev/null and b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000010.checkpoint.parquet differ diff --git a/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000010.json b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000010.json new file mode 100644 index 0000000000..c1f8c8b095 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000010.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317342605,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":9,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"ba4b200d-e0ac-4715-ad4e-bed8ef1b20cd"}} +{"add":{"path":"part-00001-55fbea2e-0788-438a-a50c-65f809acc05b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":90},\"maxValues\":{\"id\":90},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-0d891eaf-0e01-46a6-879e-49bbca90c215-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":91},\"maxValues\":{\"id\":91},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-569d12e9-04d5-4fe2-9554-f288047f3386-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342592,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":92},\"maxValues\":{\"id\":92},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-1bc89ec9-8092-49e1-9b1f-123ae50e3d40-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342579,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":93},\"maxValues\":{\"id\":93},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-b4a223d6-1d87-49c9-84c9-a85eece61839-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342591,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":94},\"maxValues\":{\"id\":94},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-4c3b6be7-979c-4f42-8920-efa32b751d97-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":95},\"maxValues\":{\"id\":95},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-6c224359-8995-417a-8b24-b2e530327bc6-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":96},\"maxValues\":{\"id\":96},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-2a40eb21-34d2-48ca-aaa5-55db674f56de-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342590,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":97},\"maxValues\":{\"id\":97},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-ca8256ed-98cd-460d-8de2-9f6f7f388703-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342579,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":98},\"maxValues\":{\"id\":98},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-f131fc78-c201-4e8d-b194-222b2e79778d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":99},\"maxValues\":{\"id\":99},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000011.json b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000011.json new file mode 100644 index 0000000000..05863f6cd0 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000011.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317349152,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":10,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"1657fc30-879d-4b0b-972a-4e3a079fdd7a"}} +{"add":{"path":"part-00001-ceaadd5e-615b-455d-8f4b-052b9c94c7b6-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349136,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":100},\"maxValues\":{\"id\":100},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-b7dba1e7-b1e5-4f02-a223-69ec7353ab45-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349123,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":101},\"maxValues\":{\"id\":101},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-f099cf4d-d418-4852-8580-091908847a66-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317349122,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":102},\"maxValues\":{\"id\":102},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-418e8d25-7316-442a-9bc8-616ed01231eb-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349136,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":103},\"maxValues\":{\"id\":103},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-7538a9c2-1ccb-4150-b162-ef8d826fe30f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349123,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":104},\"maxValues\":{\"id\":104},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-7619f42c-5bc4-4e77-b037-f36481c8b63c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349123,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":105},\"maxValues\":{\"id\":105},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-eb49d85f-91cc-4293-9339-a664ee905b0f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349134,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":106},\"maxValues\":{\"id\":106},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-81e22719-7705-4703-b2dd-c4e2982217a7-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349122,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":107},\"maxValues\":{\"id\":107},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-883858d1-9df6-4b55-a2be-5b8387134617-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349122,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":108},\"maxValues\":{\"id\":108},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-07d91938-ac89-48cc-a657-6067d2d9f67e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349137,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":109},\"maxValues\":{\"id\":109},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000012.json b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000012.json new file mode 100644 index 0000000000..4cc44fa8e8 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000012.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317349950,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":11,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"9a035bdd-f892-4449-9c39-401f31fcada6"}} +{"add":{"path":"part-00001-f3b19100-b5b3-4e72-8658-7a937e9ed515-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349924,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":110},\"maxValues\":{\"id\":110},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-54f2324a-e97f-4def-9101-9cc10599ba06-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349919,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":111},\"maxValues\":{\"id\":111},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-3f7ca40a-6497-4208-8a1a-11062456a5a9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349936,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":112},\"maxValues\":{\"id\":112},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-a14852b2-c743-4a4a-b9c1-0c9472c51699-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349929,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":113},\"maxValues\":{\"id\":113},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-97d06207-5584-43df-afc2-2d1738d79193-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349943,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":114},\"maxValues\":{\"id\":114},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-0d431f03-6dbf-40e7-96fc-b1ebbbe9fc65-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349922,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":115},\"maxValues\":{\"id\":115},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-af0f0232-33c8-4315-821b-8bb1323b7a26-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317349936,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":116},\"maxValues\":{\"id\":116},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-4f744428-d088-497e-afd3-0b374e453e7c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349936,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":117},\"maxValues\":{\"id\":117},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-694064b8-137e-45cd-b2ea-e28af172a2dc-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349918,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":118},\"maxValues\":{\"id\":118},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-56073753-4c1c-4a68-9b4a-13ef5d1a75fb-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349938,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":119},\"maxValues\":{\"id\":119},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000013.json b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000013.json new file mode 100644 index 0000000000..b2d03d3ead --- /dev/null +++ b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000000013.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317350712,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":12,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"c3cd0fa1-9c72-4344-8225-0b787e52d5e0"}} +{"add":{"path":"part-00001-7a0d95f8-e122-4cf6-b89c-389036a9b415-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350696,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":120},\"maxValues\":{\"id\":120},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-f1f035c1-bf0f-485c-950d-c81d0d2aa8a2-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350706,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":121},\"maxValues\":{\"id\":121},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-d7a51e45-70f3-4379-819b-341951abefff-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350691,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":122},\"maxValues\":{\"id\":122},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-4828722c-5799-4be1-ace1-14bd7f477dbf-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350691,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":123},\"maxValues\":{\"id\":123},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-f3c3c72e-5d71-4dc9-9e15-342f1d6cb6cc-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350701,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":124},\"maxValues\":{\"id\":124},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-e8d74ede-8876-4f55-8e9f-1bbde0d07a35-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350696,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":125},\"maxValues\":{\"id\":125},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-91052146-2292-45c3-b57e-1fd2dd6be6ed-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350692,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":126},\"maxValues\":{\"id\":126},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-9d203964-9f4c-4c84-ad77-9ba305bb6572-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350706,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":127},\"maxValues\":{\"id\":127},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-8362228b-acf6-4937-875b-26c013c342e1-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350690,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":128},\"maxValues\":{\"id\":128},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-6f57658e-1953-4b59-b504-27c9e8c5cc3b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350677,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":129},\"maxValues\":{\"id\":129},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000009999.checkpoint.parquet b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000009999.checkpoint.parquet new file mode 100644 index 0000000000..9df253f2f7 Binary files /dev/null and b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000009999.checkpoint.parquet differ diff --git a/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000009999.json b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000009999.json new file mode 100644 index 0000000000..c1f8c8b095 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_g/_delta_log/00000000000000009999.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317342605,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":9,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"ba4b200d-e0ac-4715-ad4e-bed8ef1b20cd"}} +{"add":{"path":"part-00001-55fbea2e-0788-438a-a50c-65f809acc05b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":90},\"maxValues\":{\"id\":90},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-0d891eaf-0e01-46a6-879e-49bbca90c215-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":91},\"maxValues\":{\"id\":91},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-569d12e9-04d5-4fe2-9554-f288047f3386-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342592,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":92},\"maxValues\":{\"id\":92},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-1bc89ec9-8092-49e1-9b1f-123ae50e3d40-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342579,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":93},\"maxValues\":{\"id\":93},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-b4a223d6-1d87-49c9-84c9-a85eece61839-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342591,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":94},\"maxValues\":{\"id\":94},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-4c3b6be7-979c-4f42-8920-efa32b751d97-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":95},\"maxValues\":{\"id\":95},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-6c224359-8995-417a-8b24-b2e530327bc6-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":96},\"maxValues\":{\"id\":96},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-2a40eb21-34d2-48ca-aaa5-55db674f56de-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342590,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":97},\"maxValues\":{\"id\":97},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-ca8256ed-98cd-460d-8de2-9f6f7f388703-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342579,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":98},\"maxValues\":{\"id\":98},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-f131fc78-c201-4e8d-b194-222b2e79778d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":99},\"maxValues\":{\"id\":99},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_g/_delta_log/_last_checkpoint b/crates/core/tests/data_err_logs/table_g/_delta_log/_last_checkpoint new file mode 100644 index 0000000000..604c2b31a2 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_g/_delta_log/_last_checkpoint @@ -0,0 +1 @@ +{"version":10,"size":102,"sizeInBytes":21615,"numOfAddFiles":100,"checkpointSchema":{"type":"struct","fields":[{"name":"txn","type":{"type":"struct","fields":[{"name":"appId","type":"string","nullable":true,"metadata":{}},{"name":"version","type":"long","nullable":true,"metadata":{}},{"name":"lastUpdated","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"add","type":{"type":"struct","fields":[{"name":"path","type":"string","nullable":true,"metadata":{}},{"name":"partitionValues","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"size","type":"long","nullable":true,"metadata":{}},{"name":"modificationTime","type":"long","nullable":true,"metadata":{}},{"name":"dataChange","type":"boolean","nullable":true,"metadata":{}},{"name":"tags","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"deletionVector","type":{"type":"struct","fields":[{"name":"storageType","type":"string","nullable":true,"metadata":{}},{"name":"pathOrInlineDv","type":"string","nullable":true,"metadata":{}},{"name":"offset","type":"integer","nullable":true,"metadata":{}},{"name":"sizeInBytes","type":"integer","nullable":true,"metadata":{}},{"name":"cardinality","type":"long","nullable":true,"metadata":{}},{"name":"maxRowIndex","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"baseRowId","type":"long","nullable":true,"metadata":{}},{"name":"defaultRowCommitVersion","type":"long","nullable":true,"metadata":{}},{"name":"clusteringProvider","type":"string","nullable":true,"metadata":{}},{"name":"stats","type":"string","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"remove","type":{"type":"struct","fields":[{"name":"path","type":"string","nullable":true,"metadata":{}},{"name":"deletionTimestamp","type":"long","nullable":true,"metadata":{}},{"name":"dataChange","type":"boolean","nullable":true,"metadata":{}},{"name":"extendedFileMetadata","type":"boolean","nullable":true,"metadata":{}},{"name":"partitionValues","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"size","type":"long","nullable":true,"metadata":{}},{"name":"deletionVector","type":{"type":"struct","fields":[{"name":"storageType","type":"string","nullable":true,"metadata":{}},{"name":"pathOrInlineDv","type":"string","nullable":true,"metadata":{}},{"name":"offset","type":"integer","nullable":true,"metadata":{}},{"name":"sizeInBytes","type":"integer","nullable":true,"metadata":{}},{"name":"cardinality","type":"long","nullable":true,"metadata":{}},{"name":"maxRowIndex","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"baseRowId","type":"long","nullable":true,"metadata":{}},{"name":"defaultRowCommitVersion","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"metaData","type":{"type":"struct","fields":[{"name":"id","type":"string","nullable":true,"metadata":{}},{"name":"name","type":"string","nullable":true,"metadata":{}},{"name":"description","type":"string","nullable":true,"metadata":{}},{"name":"format","type":{"type":"struct","fields":[{"name":"provider","type":"string","nullable":true,"metadata":{}},{"name":"options","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"schemaString","type":"string","nullable":true,"metadata":{}},{"name":"partitionColumns","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"configuration","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"createdTime","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"protocol","type":{"type":"struct","fields":[{"name":"minReaderVersion","type":"integer","nullable":true,"metadata":{}},{"name":"minWriterVersion","type":"integer","nullable":true,"metadata":{}},{"name":"readerFeatures","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"writerFeatures","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"domainMetadata","type":{"type":"struct","fields":[{"name":"domain","type":"string","nullable":true,"metadata":{}},{"name":"configuration","type":"string","nullable":true,"metadata":{}},{"name":"removed","type":"boolean","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}}]},"checksum":"94a578f92841fa7ba9cdee96b5905fdb"} diff --git a/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000000.json b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000000.json new file mode 100644 index 0000000000..0316f09771 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000000.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1742317262289,"operation":"CREATE TABLE","operationParameters":{"partitionBy":"[]","clusterBy":"[]","description":null,"isManaged":"false","properties":"{}"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"c0983a7a-aeca-4ba8-a509-d5dbc71a10de"}} +{"metaData":{"id":"5f54ef5f-e511-4114-b6e5-f6c206c068b6","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1742317261939}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":1}} diff --git a/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000001.json b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000001.json new file mode 100644 index 0000000000..23002c6f58 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000001.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317316973,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"a1c49e36-8fcd-46e6-8ede-5b8560f7ec3b"}} +{"add":{"path":"part-00001-665397f5-1435-4478-a598-ca226c99ffcf-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316825,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":0},\"maxValues\":{\"id\":0},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-fd50becf-074e-4a1f-985b-01529e9f7b03-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316820,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1},\"maxValues\":{\"id\":1},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-4fc1f70f-9daa-46e6-83b5-ea8144d4a96d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316822,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":2},\"maxValues\":{\"id\":2},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-efb0808c-3b7f-4a4d-bc36-daa91c074b5b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316825,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":3},\"maxValues\":{\"id\":3},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-c8664e02-01fe-4c2d-8eba-ae84012d7aad-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316825,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":4},\"maxValues\":{\"id\":4},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-7446ef47-3110-4c3f-a2d0-0c71bafc893a-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316820,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":5},\"maxValues\":{\"id\":5},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-ee372cff-6aae-4979-970b-88cc154a31bd-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316822,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":6},\"maxValues\":{\"id\":6},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-9ea59fdf-fc26-4650-a282-9c2cc1906c7c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316823,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":7},\"maxValues\":{\"id\":7},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-d50ebfbf-e534-4bc8-b63d-437f6029da6e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316823,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":8},\"maxValues\":{\"id\":8},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-96e67376-3d15-4895-bd5d-5e0a325bcb83-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316823,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":9},\"maxValues\":{\"id\":9},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000002.json b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000002.json new file mode 100644 index 0000000000..86e3c7470c --- /dev/null +++ b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000002.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317326453,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":1,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"ac59851b-981e-4e82-96ea-36a543cfe254"}} +{"add":{"path":"part-00001-6491d41d-d498-4a89-a291-92d964035606-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":10},\"maxValues\":{\"id\":10},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-d359921a-3cb1-454d-8aa0-ac5c830fcdc5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":11},\"maxValues\":{\"id\":11},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-b2aae64d-1fab-4106-bc87-2454e945dada-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":12},\"maxValues\":{\"id\":12},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-cca989ea-d56e-4e1e-a4ba-538ef7801997-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":13},\"maxValues\":{\"id\":13},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-14706643-f3f0-4ba9-8282-7d55bb4ecacb-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":14},\"maxValues\":{\"id\":14},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-25801ed5-1cf2-43fa-bbdb-8898fc102e64-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":15},\"maxValues\":{\"id\":15},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-03fbeb6f-b8a5-448e-afa7-0f49fca61866-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":16},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-08361c76-870e-4ddf-9153-f67852849ec3-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":17},\"maxValues\":{\"id\":17},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-368e738e-0673-4e76-a1ff-5ba9c755396e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":18},\"maxValues\":{\"id\":18},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-2c9a3837-e2c5-42bd-b888-f3205f4b894c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":19},\"maxValues\":{\"id\":19},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000003.json b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000003.json new file mode 100644 index 0000000000..14665bcd27 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000003.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317330682,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":2,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"5e6fdfee-2a82-40d1-bc83-47b3f0f3f21e"}} +{"add":{"path":"part-00001-b7be8377-b715-4234-b316-201fd2c9c142-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":20},\"maxValues\":{\"id\":20},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-719b1086-b19c-45d1-8c4c-c11db02e2e0b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":21},\"maxValues\":{\"id\":21},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-eb0fd03c-3e47-42ed-9897-e79dd1567fb1-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":22},\"maxValues\":{\"id\":22},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-48b99dee-6d3e-4cfb-b651-4769de7f5b24-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":23},\"maxValues\":{\"id\":23},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-b873a231-8352-4bac-b6f1-b53ee738d212-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330582,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":24},\"maxValues\":{\"id\":24},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-74ed051c-b116-4947-b62c-2086bcd5bb90-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":25},\"maxValues\":{\"id\":25},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-680d9e49-afce-4a7d-bca8-b03438c2fd74-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":26},\"maxValues\":{\"id\":26},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-81c10052-aacc-4ecf-b9cf-64f81b3bd435-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":27},\"maxValues\":{\"id\":27},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-cf9cac69-932c-43bf-8e4b-fd059d519c0f-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":28},\"maxValues\":{\"id\":28},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-43e3252c-8ac7-4c7a-bcb4-15aaf7ae95b9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":29},\"maxValues\":{\"id\":29},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000004.json b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000004.json new file mode 100644 index 0000000000..ed86e283d5 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000004.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317333588,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":3,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"57611e9e-5312-4502-a3ad-c0c78799773e"}} +{"add":{"path":"part-00001-5c92b4bb-af84-4066-8aeb-1e493b7147df-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333524,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":30},\"maxValues\":{\"id\":30},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-b050a084-ab25-420f-bb7b-50eb95d25e4e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333551,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":31},\"maxValues\":{\"id\":31},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-1880b504-fcf5-4f43-92d4-c43e8dd9d7d9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333550,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":32},\"maxValues\":{\"id\":32},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-4af9173b-8472-41cd-8772-e2bdb084c5d5-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317333524,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":33},\"maxValues\":{\"id\":33},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-89cec234-f844-4802-a786-5d9133bbe489-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333523,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":34},\"maxValues\":{\"id\":34},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-fa11b282-ec0d-4513-9baf-2b84c5f94a12-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333550,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":35},\"maxValues\":{\"id\":35},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-895723cb-0dba-4019-a2a9-e6db9a937c91-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333550,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":36},\"maxValues\":{\"id\":36},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-dca57e7a-f859-4b39-bc43-03e1061f1b4e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333551,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":37},\"maxValues\":{\"id\":37},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-96756753-7714-4c07-a238-d5b57f42a8ce-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333523,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":38},\"maxValues\":{\"id\":38},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-9a7b861f-5d9a-41c4-b4ec-7f0d1391acfe-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333551,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":39},\"maxValues\":{\"id\":39},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000005.json b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000005.json new file mode 100644 index 0000000000..f77fd655fe --- /dev/null +++ b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000005.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317336099,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":4,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"203edd74-d2b2-46fe-935a-6222cfb888d4"}} +{"add":{"path":"part-00001-a24fe71c-ba35-47bb-8f3e-636d5991d5ae-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336058,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":40},\"maxValues\":{\"id\":40},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-a56ad50f-bc64-44cb-bb55-e2d177947b3d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336079,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":41},\"maxValues\":{\"id\":41},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-95c53cf1-b472-4c34-b728-1dd7cbed8b2f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336074,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":42},\"maxValues\":{\"id\":42},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-7e442ced-e810-44d9-9d28-3027e652a0ec-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336080,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":43},\"maxValues\":{\"id\":43},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-68248457-2fa3-407e-9de3-759b1e052b99-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336075,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":44},\"maxValues\":{\"id\":44},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-89a48ead-5bf3-4d16-aada-97c11386fcaf-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336076,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":45},\"maxValues\":{\"id\":45},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-42b618de-c46c-4888-9b48-b99493ec2983-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336070,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":46},\"maxValues\":{\"id\":46},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-4dc49305-f4f8-4ec9-9a40-8f4b3bd81324-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336055,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":47},\"maxValues\":{\"id\":47},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-d7bb17a9-223e-474b-9d78-2c745cc35a4b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336054,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":48},\"maxValues\":{\"id\":48},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-68a79bb6-a31e-49bf-848f-2d64ceb834c0-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336079,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":49},\"maxValues\":{\"id\":49},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000006.json b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000006.json new file mode 100644 index 0000000000..28116ba9f2 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000006.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317338700,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":5,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"9ee4b86b-1bbe-4d6e-adbd-8dd4961989fb"}} +{"add":{"path":"part-00001-a8fc5b00-29e4-4a99-961d-b0cbcc23d165-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338678,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":50},\"maxValues\":{\"id\":50},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-5a3079f8-abbc-4b5f-a1e3-340830e59222-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317338670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":51},\"maxValues\":{\"id\":51},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-567d7b20-b6ce-4e96-b500-caa34c80f8a7-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338676,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":52},\"maxValues\":{\"id\":52},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-b794df4b-174f-468a-9de7-2aa865ba7014-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338676,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":53},\"maxValues\":{\"id\":53},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-a15406cf-c141-4f7b-b302-e4b5a145cad5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338675,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":54},\"maxValues\":{\"id\":54},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-8a52349c-d93b-4c59-b493-13486bb5e284-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338680,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":55},\"maxValues\":{\"id\":55},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-1734b4eb-4414-4b3a-8e99-1bd099c9e6b5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":56},\"maxValues\":{\"id\":56},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-02781f2a-6c34-42ca-80a4-e830b2eeb963-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338676,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":57},\"maxValues\":{\"id\":57},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-63ec1c21-c31c-43d4-b5c9-9c206aeeb280-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":58},\"maxValues\":{\"id\":58},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-b732e8e4-7d1b-470d-89a5-86a3f8d8bdc2-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338680,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":59},\"maxValues\":{\"id\":59},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000007.json b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000007.json new file mode 100644 index 0000000000..956c8b508c --- /dev/null +++ b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000007.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317339658,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":6,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"63c885b0-74eb-4075-a02a-a43b8202b3f8"}} +{"add":{"path":"part-00001-f697bc51-b77f-4234-938a-5f85478cedec-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":60},\"maxValues\":{\"id\":60},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-839f210e-cf84-4c5c-b185-fd2fe2b5ee6f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339491,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":61},\"maxValues\":{\"id\":61},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-9ebe1c22-87a1-4f37-a695-77658c3e70a8-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339491,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":62},\"maxValues\":{\"id\":62},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-01b2687c-45e4-484c-b1d6-80e06b5b5d11-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339490,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":63},\"maxValues\":{\"id\":63},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-05662cc4-6a79-4204-aec1-2311a44d8c74-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339511,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":64},\"maxValues\":{\"id\":64},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-67f1b806-ef5f-4f8a-890b-b3b5ad1d234c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339490,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":65},\"maxValues\":{\"id\":65},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-37318455-4128-4e1e-9ab7-5c587ac9fde0-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":66},\"maxValues\":{\"id\":66},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-c5d68934-1f5a-40c4-b5be-1233eb15378a-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339511,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":67},\"maxValues\":{\"id\":67},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-41e40903-13b6-4465-aa3c-bd8cb5e52b18-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":68},\"maxValues\":{\"id\":68},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-110c626e-ea13-4204-8cae-a3183d89a4b7-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":69},\"maxValues\":{\"id\":69},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000008.json b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000008.json new file mode 100644 index 0000000000..ea216c1556 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000008.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317340794,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":7,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"b5902ae9-2aab-46f9-82f9-d68ae45940d7"}} +{"add":{"path":"part-00001-3b62f1d4-2a3e-4611-a55b-e9d2ace11b3c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340752,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":70},\"maxValues\":{\"id\":70},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-9c066923-23f4-45f6-b2af-5a4ecbef1707-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340776,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":71},\"maxValues\":{\"id\":71},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-cfb48b6e-0fc7-4d6b-8ab7-c52f29f71b94-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340752,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":72},\"maxValues\":{\"id\":72},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-b21a45a5-9a53-4dfa-8327-8a82b6b283e9-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317340766,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":73},\"maxValues\":{\"id\":73},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-afa450c4-f649-4c88-817a-6d0bdfc4da6f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340765,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":74},\"maxValues\":{\"id\":74},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-dd96cd25-394d-4873-84e7-f2f6b0eb5a67-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340776,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":75},\"maxValues\":{\"id\":75},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-3a98b7ed-8665-4bc5-8704-6745f7084cd0-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340776,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":76},\"maxValues\":{\"id\":76},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-8dfd3dc5-cf31-42fc-8c55-2ac70ce9e18d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340782,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":77},\"maxValues\":{\"id\":77},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-97e7d287-aee3-445d-a90e-f3b2ef4bd7cd-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340765,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":78},\"maxValues\":{\"id\":78},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-db2340aa-28ff-4826-b39e-07ba516551e9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340779,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":79},\"maxValues\":{\"id\":79},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000009.json b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000009.json new file mode 100644 index 0000000000..1f9d5fb05b --- /dev/null +++ b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000009.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317341714,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":8,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"5379512e-d4d3-42b5-817d-70ecf05f2385"}} +{"add":{"path":"part-00001-4e7175fd-6ffb-4b6a-946c-43aa7c439104-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":80},\"maxValues\":{\"id\":80},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-f3cc4f07-93ec-4a47-add1-b16c1149c3d9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341667,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":81},\"maxValues\":{\"id\":81},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-07581f2d-ee98-4464-a28b-f738e88749e4-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":82},\"maxValues\":{\"id\":82},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-c9d7468d-fc14-445c-8dbd-65d616f8eb05-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341666,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":83},\"maxValues\":{\"id\":83},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-56eb2260-fb49-4138-a5c8-f0ae0949f4e2-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":84},\"maxValues\":{\"id\":84},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-5d15eeca-0fca-4986-a18e-4d86bf5ba2f6-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":85},\"maxValues\":{\"id\":85},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-3e13d733-f55d-42ac-be4f-f4400e999c29-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":86},\"maxValues\":{\"id\":86},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-22c23f1e-26d2-488e-8e07-2de6ae5fded5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341667,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":87},\"maxValues\":{\"id\":87},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-d0debf09-5f82-4c61-8636-27e51fba37e5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341674,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":88},\"maxValues\":{\"id\":88},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-7403e0c4-bb07-4a1b-9fca-a01523713f85-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":89},\"maxValues\":{\"id\":89},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000010.checkpoint.parquet b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000010.checkpoint.parquet new file mode 100644 index 0000000000..9df253f2f7 Binary files /dev/null and b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000010.checkpoint.parquet differ diff --git a/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000010.json b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000010.json new file mode 100644 index 0000000000..c1f8c8b095 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000010.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317342605,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":9,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"ba4b200d-e0ac-4715-ad4e-bed8ef1b20cd"}} +{"add":{"path":"part-00001-55fbea2e-0788-438a-a50c-65f809acc05b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":90},\"maxValues\":{\"id\":90},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-0d891eaf-0e01-46a6-879e-49bbca90c215-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":91},\"maxValues\":{\"id\":91},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-569d12e9-04d5-4fe2-9554-f288047f3386-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342592,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":92},\"maxValues\":{\"id\":92},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-1bc89ec9-8092-49e1-9b1f-123ae50e3d40-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342579,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":93},\"maxValues\":{\"id\":93},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-b4a223d6-1d87-49c9-84c9-a85eece61839-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342591,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":94},\"maxValues\":{\"id\":94},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-4c3b6be7-979c-4f42-8920-efa32b751d97-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":95},\"maxValues\":{\"id\":95},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-6c224359-8995-417a-8b24-b2e530327bc6-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":96},\"maxValues\":{\"id\":96},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-2a40eb21-34d2-48ca-aaa5-55db674f56de-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342590,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":97},\"maxValues\":{\"id\":97},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-ca8256ed-98cd-460d-8de2-9f6f7f388703-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342579,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":98},\"maxValues\":{\"id\":98},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-f131fc78-c201-4e8d-b194-222b2e79778d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":99},\"maxValues\":{\"id\":99},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000011.json b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000011.json new file mode 100644 index 0000000000..05863f6cd0 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000011.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317349152,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":10,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"1657fc30-879d-4b0b-972a-4e3a079fdd7a"}} +{"add":{"path":"part-00001-ceaadd5e-615b-455d-8f4b-052b9c94c7b6-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349136,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":100},\"maxValues\":{\"id\":100},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-b7dba1e7-b1e5-4f02-a223-69ec7353ab45-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349123,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":101},\"maxValues\":{\"id\":101},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-f099cf4d-d418-4852-8580-091908847a66-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317349122,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":102},\"maxValues\":{\"id\":102},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-418e8d25-7316-442a-9bc8-616ed01231eb-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349136,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":103},\"maxValues\":{\"id\":103},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-7538a9c2-1ccb-4150-b162-ef8d826fe30f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349123,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":104},\"maxValues\":{\"id\":104},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-7619f42c-5bc4-4e77-b037-f36481c8b63c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349123,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":105},\"maxValues\":{\"id\":105},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-eb49d85f-91cc-4293-9339-a664ee905b0f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349134,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":106},\"maxValues\":{\"id\":106},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-81e22719-7705-4703-b2dd-c4e2982217a7-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349122,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":107},\"maxValues\":{\"id\":107},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-883858d1-9df6-4b55-a2be-5b8387134617-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349122,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":108},\"maxValues\":{\"id\":108},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-07d91938-ac89-48cc-a657-6067d2d9f67e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349137,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":109},\"maxValues\":{\"id\":109},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000012.json b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000012.json new file mode 100644 index 0000000000..4cc44fa8e8 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000012.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317349950,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":11,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"9a035bdd-f892-4449-9c39-401f31fcada6"}} +{"add":{"path":"part-00001-f3b19100-b5b3-4e72-8658-7a937e9ed515-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349924,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":110},\"maxValues\":{\"id\":110},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-54f2324a-e97f-4def-9101-9cc10599ba06-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349919,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":111},\"maxValues\":{\"id\":111},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-3f7ca40a-6497-4208-8a1a-11062456a5a9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349936,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":112},\"maxValues\":{\"id\":112},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-a14852b2-c743-4a4a-b9c1-0c9472c51699-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349929,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":113},\"maxValues\":{\"id\":113},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-97d06207-5584-43df-afc2-2d1738d79193-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349943,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":114},\"maxValues\":{\"id\":114},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-0d431f03-6dbf-40e7-96fc-b1ebbbe9fc65-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349922,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":115},\"maxValues\":{\"id\":115},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-af0f0232-33c8-4315-821b-8bb1323b7a26-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317349936,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":116},\"maxValues\":{\"id\":116},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-4f744428-d088-497e-afd3-0b374e453e7c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349936,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":117},\"maxValues\":{\"id\":117},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-694064b8-137e-45cd-b2ea-e28af172a2dc-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349918,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":118},\"maxValues\":{\"id\":118},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-56073753-4c1c-4a68-9b4a-13ef5d1a75fb-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349938,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":119},\"maxValues\":{\"id\":119},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000013.json b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000013.json new file mode 100644 index 0000000000..b2d03d3ead --- /dev/null +++ b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000000013.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317350712,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":12,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"c3cd0fa1-9c72-4344-8225-0b787e52d5e0"}} +{"add":{"path":"part-00001-7a0d95f8-e122-4cf6-b89c-389036a9b415-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350696,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":120},\"maxValues\":{\"id\":120},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-f1f035c1-bf0f-485c-950d-c81d0d2aa8a2-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350706,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":121},\"maxValues\":{\"id\":121},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-d7a51e45-70f3-4379-819b-341951abefff-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350691,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":122},\"maxValues\":{\"id\":122},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-4828722c-5799-4be1-ace1-14bd7f477dbf-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350691,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":123},\"maxValues\":{\"id\":123},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-f3c3c72e-5d71-4dc9-9e15-342f1d6cb6cc-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350701,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":124},\"maxValues\":{\"id\":124},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-e8d74ede-8876-4f55-8e9f-1bbde0d07a35-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350696,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":125},\"maxValues\":{\"id\":125},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-91052146-2292-45c3-b57e-1fd2dd6be6ed-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350692,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":126},\"maxValues\":{\"id\":126},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-9d203964-9f4c-4c84-ad77-9ba305bb6572-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350706,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":127},\"maxValues\":{\"id\":127},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-8362228b-acf6-4937-875b-26c013c342e1-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350690,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":128},\"maxValues\":{\"id\":128},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-6f57658e-1953-4b59-b504-27c9e8c5cc3b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350677,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":129},\"maxValues\":{\"id\":129},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000009999.checkpoint.parquet b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000009999.checkpoint.parquet new file mode 100644 index 0000000000..9fd1755fd5 Binary files /dev/null and b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000009999.checkpoint.parquet differ diff --git a/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000009999.json b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000009999.json new file mode 100644 index 0000000000..d78fe64124 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_h/_delta_log/00000000000000009999.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1742252106605,"engineInfo":"Kernel-3.4.0-SNAPSHOT/test-engine","operation":"CREATE TABLE","operationParameters":{"partitionBy":"[]"},"isBlindAppend":true,"txnId":"138db460-c7d3-4d13-972e-0f4e35b58b43","operationMetrics":{}}} +{"metaData":{"id":"ebffcf7b-bf25-427f-a91a-d4a0d27ebbef","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"createdTime":1742252106422,"configuration":{"delta.feature.catalogowned":"supported"}}} +{"protocol":{"minReaderVersion":3,"minWriterVersion":7,"readerFeatures":["catalogOwned"],"writerFeatures":["catalogOwned","invariants","appendOnly"]}} diff --git a/crates/core/tests/data_err_logs/table_h/_delta_log/_last_checkpoint b/crates/core/tests/data_err_logs/table_h/_delta_log/_last_checkpoint new file mode 100644 index 0000000000..604c2b31a2 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_h/_delta_log/_last_checkpoint @@ -0,0 +1 @@ +{"version":10,"size":102,"sizeInBytes":21615,"numOfAddFiles":100,"checkpointSchema":{"type":"struct","fields":[{"name":"txn","type":{"type":"struct","fields":[{"name":"appId","type":"string","nullable":true,"metadata":{}},{"name":"version","type":"long","nullable":true,"metadata":{}},{"name":"lastUpdated","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"add","type":{"type":"struct","fields":[{"name":"path","type":"string","nullable":true,"metadata":{}},{"name":"partitionValues","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"size","type":"long","nullable":true,"metadata":{}},{"name":"modificationTime","type":"long","nullable":true,"metadata":{}},{"name":"dataChange","type":"boolean","nullable":true,"metadata":{}},{"name":"tags","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"deletionVector","type":{"type":"struct","fields":[{"name":"storageType","type":"string","nullable":true,"metadata":{}},{"name":"pathOrInlineDv","type":"string","nullable":true,"metadata":{}},{"name":"offset","type":"integer","nullable":true,"metadata":{}},{"name":"sizeInBytes","type":"integer","nullable":true,"metadata":{}},{"name":"cardinality","type":"long","nullable":true,"metadata":{}},{"name":"maxRowIndex","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"baseRowId","type":"long","nullable":true,"metadata":{}},{"name":"defaultRowCommitVersion","type":"long","nullable":true,"metadata":{}},{"name":"clusteringProvider","type":"string","nullable":true,"metadata":{}},{"name":"stats","type":"string","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"remove","type":{"type":"struct","fields":[{"name":"path","type":"string","nullable":true,"metadata":{}},{"name":"deletionTimestamp","type":"long","nullable":true,"metadata":{}},{"name":"dataChange","type":"boolean","nullable":true,"metadata":{}},{"name":"extendedFileMetadata","type":"boolean","nullable":true,"metadata":{}},{"name":"partitionValues","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"size","type":"long","nullable":true,"metadata":{}},{"name":"deletionVector","type":{"type":"struct","fields":[{"name":"storageType","type":"string","nullable":true,"metadata":{}},{"name":"pathOrInlineDv","type":"string","nullable":true,"metadata":{}},{"name":"offset","type":"integer","nullable":true,"metadata":{}},{"name":"sizeInBytes","type":"integer","nullable":true,"metadata":{}},{"name":"cardinality","type":"long","nullable":true,"metadata":{}},{"name":"maxRowIndex","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"baseRowId","type":"long","nullable":true,"metadata":{}},{"name":"defaultRowCommitVersion","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"metaData","type":{"type":"struct","fields":[{"name":"id","type":"string","nullable":true,"metadata":{}},{"name":"name","type":"string","nullable":true,"metadata":{}},{"name":"description","type":"string","nullable":true,"metadata":{}},{"name":"format","type":{"type":"struct","fields":[{"name":"provider","type":"string","nullable":true,"metadata":{}},{"name":"options","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"schemaString","type":"string","nullable":true,"metadata":{}},{"name":"partitionColumns","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"configuration","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"createdTime","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"protocol","type":{"type":"struct","fields":[{"name":"minReaderVersion","type":"integer","nullable":true,"metadata":{}},{"name":"minWriterVersion","type":"integer","nullable":true,"metadata":{}},{"name":"readerFeatures","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"writerFeatures","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"domainMetadata","type":{"type":"struct","fields":[{"name":"domain","type":"string","nullable":true,"metadata":{}},{"name":"configuration","type":"string","nullable":true,"metadata":{}},{"name":"removed","type":"boolean","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}}]},"checksum":"94a578f92841fa7ba9cdee96b5905fdb"} diff --git a/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000000.json b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000000.json new file mode 100644 index 0000000000..0316f09771 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000000.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1742317262289,"operation":"CREATE TABLE","operationParameters":{"partitionBy":"[]","clusterBy":"[]","description":null,"isManaged":"false","properties":"{}"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"c0983a7a-aeca-4ba8-a509-d5dbc71a10de"}} +{"metaData":{"id":"5f54ef5f-e511-4114-b6e5-f6c206c068b6","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1742317261939}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":1}} diff --git a/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000001.json b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000001.json new file mode 100644 index 0000000000..23002c6f58 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000001.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317316973,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"a1c49e36-8fcd-46e6-8ede-5b8560f7ec3b"}} +{"add":{"path":"part-00001-665397f5-1435-4478-a598-ca226c99ffcf-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316825,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":0},\"maxValues\":{\"id\":0},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-fd50becf-074e-4a1f-985b-01529e9f7b03-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316820,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1},\"maxValues\":{\"id\":1},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-4fc1f70f-9daa-46e6-83b5-ea8144d4a96d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316822,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":2},\"maxValues\":{\"id\":2},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-efb0808c-3b7f-4a4d-bc36-daa91c074b5b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316825,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":3},\"maxValues\":{\"id\":3},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-c8664e02-01fe-4c2d-8eba-ae84012d7aad-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316825,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":4},\"maxValues\":{\"id\":4},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-7446ef47-3110-4c3f-a2d0-0c71bafc893a-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316820,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":5},\"maxValues\":{\"id\":5},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-ee372cff-6aae-4979-970b-88cc154a31bd-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316822,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":6},\"maxValues\":{\"id\":6},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-9ea59fdf-fc26-4650-a282-9c2cc1906c7c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316823,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":7},\"maxValues\":{\"id\":7},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-d50ebfbf-e534-4bc8-b63d-437f6029da6e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316823,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":8},\"maxValues\":{\"id\":8},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-96e67376-3d15-4895-bd5d-5e0a325bcb83-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317316823,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":9},\"maxValues\":{\"id\":9},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000002.json b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000002.json new file mode 100644 index 0000000000..86e3c7470c --- /dev/null +++ b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000002.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317326453,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":1,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"ac59851b-981e-4e82-96ea-36a543cfe254"}} +{"add":{"path":"part-00001-6491d41d-d498-4a89-a291-92d964035606-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":10},\"maxValues\":{\"id\":10},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-d359921a-3cb1-454d-8aa0-ac5c830fcdc5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":11},\"maxValues\":{\"id\":11},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-b2aae64d-1fab-4106-bc87-2454e945dada-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":12},\"maxValues\":{\"id\":12},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-cca989ea-d56e-4e1e-a4ba-538ef7801997-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":13},\"maxValues\":{\"id\":13},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-14706643-f3f0-4ba9-8282-7d55bb4ecacb-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":14},\"maxValues\":{\"id\":14},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-25801ed5-1cf2-43fa-bbdb-8898fc102e64-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":15},\"maxValues\":{\"id\":15},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-03fbeb6f-b8a5-448e-afa7-0f49fca61866-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":16},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-08361c76-870e-4ddf-9153-f67852849ec3-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":17},\"maxValues\":{\"id\":17},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-368e738e-0673-4e76-a1ff-5ba9c755396e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326373,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":18},\"maxValues\":{\"id\":18},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-2c9a3837-e2c5-42bd-b888-f3205f4b894c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317326374,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":19},\"maxValues\":{\"id\":19},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000003.json b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000003.json new file mode 100644 index 0000000000..14665bcd27 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000003.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317330682,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":2,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"5e6fdfee-2a82-40d1-bc83-47b3f0f3f21e"}} +{"add":{"path":"part-00001-b7be8377-b715-4234-b316-201fd2c9c142-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":20},\"maxValues\":{\"id\":20},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-719b1086-b19c-45d1-8c4c-c11db02e2e0b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":21},\"maxValues\":{\"id\":21},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-eb0fd03c-3e47-42ed-9897-e79dd1567fb1-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":22},\"maxValues\":{\"id\":22},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-48b99dee-6d3e-4cfb-b651-4769de7f5b24-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":23},\"maxValues\":{\"id\":23},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-b873a231-8352-4bac-b6f1-b53ee738d212-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330582,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":24},\"maxValues\":{\"id\":24},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-74ed051c-b116-4947-b62c-2086bcd5bb90-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":25},\"maxValues\":{\"id\":25},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-680d9e49-afce-4a7d-bca8-b03438c2fd74-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":26},\"maxValues\":{\"id\":26},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-81c10052-aacc-4ecf-b9cf-64f81b3bd435-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":27},\"maxValues\":{\"id\":27},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-cf9cac69-932c-43bf-8e4b-fd059d519c0f-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317330577,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":28},\"maxValues\":{\"id\":28},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-43e3252c-8ac7-4c7a-bcb4-15aaf7ae95b9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317330581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":29},\"maxValues\":{\"id\":29},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000004.json b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000004.json new file mode 100644 index 0000000000..ed86e283d5 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000004.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317333588,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":3,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"57611e9e-5312-4502-a3ad-c0c78799773e"}} +{"add":{"path":"part-00001-5c92b4bb-af84-4066-8aeb-1e493b7147df-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333524,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":30},\"maxValues\":{\"id\":30},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-b050a084-ab25-420f-bb7b-50eb95d25e4e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333551,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":31},\"maxValues\":{\"id\":31},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-1880b504-fcf5-4f43-92d4-c43e8dd9d7d9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333550,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":32},\"maxValues\":{\"id\":32},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-4af9173b-8472-41cd-8772-e2bdb084c5d5-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317333524,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":33},\"maxValues\":{\"id\":33},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-89cec234-f844-4802-a786-5d9133bbe489-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333523,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":34},\"maxValues\":{\"id\":34},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-fa11b282-ec0d-4513-9baf-2b84c5f94a12-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333550,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":35},\"maxValues\":{\"id\":35},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-895723cb-0dba-4019-a2a9-e6db9a937c91-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333550,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":36},\"maxValues\":{\"id\":36},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-dca57e7a-f859-4b39-bc43-03e1061f1b4e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333551,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":37},\"maxValues\":{\"id\":37},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-96756753-7714-4c07-a238-d5b57f42a8ce-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333523,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":38},\"maxValues\":{\"id\":38},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-9a7b861f-5d9a-41c4-b4ec-7f0d1391acfe-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317333551,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":39},\"maxValues\":{\"id\":39},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000005.json b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000005.json new file mode 100644 index 0000000000..f77fd655fe --- /dev/null +++ b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000005.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317336099,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":4,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"203edd74-d2b2-46fe-935a-6222cfb888d4"}} +{"add":{"path":"part-00001-a24fe71c-ba35-47bb-8f3e-636d5991d5ae-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336058,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":40},\"maxValues\":{\"id\":40},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-a56ad50f-bc64-44cb-bb55-e2d177947b3d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336079,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":41},\"maxValues\":{\"id\":41},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-95c53cf1-b472-4c34-b728-1dd7cbed8b2f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336074,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":42},\"maxValues\":{\"id\":42},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-7e442ced-e810-44d9-9d28-3027e652a0ec-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336080,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":43},\"maxValues\":{\"id\":43},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-68248457-2fa3-407e-9de3-759b1e052b99-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336075,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":44},\"maxValues\":{\"id\":44},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-89a48ead-5bf3-4d16-aada-97c11386fcaf-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336076,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":45},\"maxValues\":{\"id\":45},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-42b618de-c46c-4888-9b48-b99493ec2983-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336070,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":46},\"maxValues\":{\"id\":46},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-4dc49305-f4f8-4ec9-9a40-8f4b3bd81324-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336055,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":47},\"maxValues\":{\"id\":47},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-d7bb17a9-223e-474b-9d78-2c745cc35a4b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336054,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":48},\"maxValues\":{\"id\":48},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-68a79bb6-a31e-49bf-848f-2d64ceb834c0-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317336079,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":49},\"maxValues\":{\"id\":49},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000006.json b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000006.json new file mode 100644 index 0000000000..28116ba9f2 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000006.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317338700,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":5,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"9ee4b86b-1bbe-4d6e-adbd-8dd4961989fb"}} +{"add":{"path":"part-00001-a8fc5b00-29e4-4a99-961d-b0cbcc23d165-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338678,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":50},\"maxValues\":{\"id\":50},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-5a3079f8-abbc-4b5f-a1e3-340830e59222-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317338670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":51},\"maxValues\":{\"id\":51},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-567d7b20-b6ce-4e96-b500-caa34c80f8a7-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338676,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":52},\"maxValues\":{\"id\":52},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-b794df4b-174f-468a-9de7-2aa865ba7014-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338676,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":53},\"maxValues\":{\"id\":53},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-a15406cf-c141-4f7b-b302-e4b5a145cad5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338675,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":54},\"maxValues\":{\"id\":54},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-8a52349c-d93b-4c59-b493-13486bb5e284-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338680,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":55},\"maxValues\":{\"id\":55},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-1734b4eb-4414-4b3a-8e99-1bd099c9e6b5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":56},\"maxValues\":{\"id\":56},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-02781f2a-6c34-42ca-80a4-e830b2eeb963-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338676,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":57},\"maxValues\":{\"id\":57},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-63ec1c21-c31c-43d4-b5c9-9c206aeeb280-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":58},\"maxValues\":{\"id\":58},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-b732e8e4-7d1b-470d-89a5-86a3f8d8bdc2-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317338680,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":59},\"maxValues\":{\"id\":59},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000007.json b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000007.json new file mode 100644 index 0000000000..956c8b508c --- /dev/null +++ b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000007.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317339658,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":6,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"63c885b0-74eb-4075-a02a-a43b8202b3f8"}} +{"add":{"path":"part-00001-f697bc51-b77f-4234-938a-5f85478cedec-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":60},\"maxValues\":{\"id\":60},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-839f210e-cf84-4c5c-b185-fd2fe2b5ee6f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339491,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":61},\"maxValues\":{\"id\":61},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-9ebe1c22-87a1-4f37-a695-77658c3e70a8-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339491,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":62},\"maxValues\":{\"id\":62},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-01b2687c-45e4-484c-b1d6-80e06b5b5d11-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339490,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":63},\"maxValues\":{\"id\":63},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-05662cc4-6a79-4204-aec1-2311a44d8c74-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339511,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":64},\"maxValues\":{\"id\":64},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-67f1b806-ef5f-4f8a-890b-b3b5ad1d234c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339490,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":65},\"maxValues\":{\"id\":65},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-37318455-4128-4e1e-9ab7-5c587ac9fde0-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":66},\"maxValues\":{\"id\":66},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-c5d68934-1f5a-40c4-b5be-1233eb15378a-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339511,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":67},\"maxValues\":{\"id\":67},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-41e40903-13b6-4465-aa3c-bd8cb5e52b18-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":68},\"maxValues\":{\"id\":68},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-110c626e-ea13-4204-8cae-a3183d89a4b7-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317339621,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":69},\"maxValues\":{\"id\":69},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000008.json b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000008.json new file mode 100644 index 0000000000..ea216c1556 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000008.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317340794,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":7,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"b5902ae9-2aab-46f9-82f9-d68ae45940d7"}} +{"add":{"path":"part-00001-3b62f1d4-2a3e-4611-a55b-e9d2ace11b3c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340752,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":70},\"maxValues\":{\"id\":70},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-9c066923-23f4-45f6-b2af-5a4ecbef1707-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340776,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":71},\"maxValues\":{\"id\":71},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-cfb48b6e-0fc7-4d6b-8ab7-c52f29f71b94-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340752,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":72},\"maxValues\":{\"id\":72},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-b21a45a5-9a53-4dfa-8327-8a82b6b283e9-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317340766,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":73},\"maxValues\":{\"id\":73},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-afa450c4-f649-4c88-817a-6d0bdfc4da6f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340765,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":74},\"maxValues\":{\"id\":74},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-dd96cd25-394d-4873-84e7-f2f6b0eb5a67-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340776,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":75},\"maxValues\":{\"id\":75},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-3a98b7ed-8665-4bc5-8704-6745f7084cd0-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340776,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":76},\"maxValues\":{\"id\":76},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-8dfd3dc5-cf31-42fc-8c55-2ac70ce9e18d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340782,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":77},\"maxValues\":{\"id\":77},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-97e7d287-aee3-445d-a90e-f3b2ef4bd7cd-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340765,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":78},\"maxValues\":{\"id\":78},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-db2340aa-28ff-4826-b39e-07ba516551e9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317340779,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":79},\"maxValues\":{\"id\":79},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000009.json b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000009.json new file mode 100644 index 0000000000..1f9d5fb05b --- /dev/null +++ b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000009.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317341714,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":8,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"5379512e-d4d3-42b5-817d-70ecf05f2385"}} +{"add":{"path":"part-00001-4e7175fd-6ffb-4b6a-946c-43aa7c439104-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":80},\"maxValues\":{\"id\":80},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-f3cc4f07-93ec-4a47-add1-b16c1149c3d9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341667,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":81},\"maxValues\":{\"id\":81},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-07581f2d-ee98-4464-a28b-f738e88749e4-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":82},\"maxValues\":{\"id\":82},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-c9d7468d-fc14-445c-8dbd-65d616f8eb05-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341666,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":83},\"maxValues\":{\"id\":83},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-56eb2260-fb49-4138-a5c8-f0ae0949f4e2-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":84},\"maxValues\":{\"id\":84},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-5d15eeca-0fca-4986-a18e-4d86bf5ba2f6-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341670,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":85},\"maxValues\":{\"id\":85},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-3e13d733-f55d-42ac-be4f-f4400e999c29-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":86},\"maxValues\":{\"id\":86},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-22c23f1e-26d2-488e-8e07-2de6ae5fded5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341667,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":87},\"maxValues\":{\"id\":87},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-d0debf09-5f82-4c61-8636-27e51fba37e5-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341674,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":88},\"maxValues\":{\"id\":88},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-7403e0c4-bb07-4a1b-9fca-a01523713f85-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317341702,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":89},\"maxValues\":{\"id\":89},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000010.checkpoint.parquet b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000010.checkpoint.parquet new file mode 100644 index 0000000000..9df253f2f7 Binary files /dev/null and b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000010.checkpoint.parquet differ diff --git a/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000010.json b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000010.json new file mode 100644 index 0000000000..c1f8c8b095 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000010.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317342605,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":9,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"ba4b200d-e0ac-4715-ad4e-bed8ef1b20cd"}} +{"add":{"path":"part-00001-55fbea2e-0788-438a-a50c-65f809acc05b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":90},\"maxValues\":{\"id\":90},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-0d891eaf-0e01-46a6-879e-49bbca90c215-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":91},\"maxValues\":{\"id\":91},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-569d12e9-04d5-4fe2-9554-f288047f3386-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342592,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":92},\"maxValues\":{\"id\":92},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-1bc89ec9-8092-49e1-9b1f-123ae50e3d40-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342579,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":93},\"maxValues\":{\"id\":93},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-b4a223d6-1d87-49c9-84c9-a85eece61839-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342591,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":94},\"maxValues\":{\"id\":94},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-4c3b6be7-979c-4f42-8920-efa32b751d97-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":95},\"maxValues\":{\"id\":95},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-6c224359-8995-417a-8b24-b2e530327bc6-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342581,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":96},\"maxValues\":{\"id\":96},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-2a40eb21-34d2-48ca-aaa5-55db674f56de-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342590,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":97},\"maxValues\":{\"id\":97},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-ca8256ed-98cd-460d-8de2-9f6f7f388703-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342579,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":98},\"maxValues\":{\"id\":98},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-f131fc78-c201-4e8d-b194-222b2e79778d-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317342578,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":99},\"maxValues\":{\"id\":99},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000011.json b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000011.json new file mode 100644 index 0000000000..05863f6cd0 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000011.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317349152,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":10,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"1657fc30-879d-4b0b-972a-4e3a079fdd7a"}} +{"add":{"path":"part-00001-ceaadd5e-615b-455d-8f4b-052b9c94c7b6-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349136,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":100},\"maxValues\":{\"id\":100},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-b7dba1e7-b1e5-4f02-a223-69ec7353ab45-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349123,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":101},\"maxValues\":{\"id\":101},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-f099cf4d-d418-4852-8580-091908847a66-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317349122,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":102},\"maxValues\":{\"id\":102},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-418e8d25-7316-442a-9bc8-616ed01231eb-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349136,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":103},\"maxValues\":{\"id\":103},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-7538a9c2-1ccb-4150-b162-ef8d826fe30f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349123,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":104},\"maxValues\":{\"id\":104},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-7619f42c-5bc4-4e77-b037-f36481c8b63c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349123,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":105},\"maxValues\":{\"id\":105},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-eb49d85f-91cc-4293-9339-a664ee905b0f-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349134,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":106},\"maxValues\":{\"id\":106},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-81e22719-7705-4703-b2dd-c4e2982217a7-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349122,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":107},\"maxValues\":{\"id\":107},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-883858d1-9df6-4b55-a2be-5b8387134617-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349122,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":108},\"maxValues\":{\"id\":108},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-07d91938-ac89-48cc-a657-6067d2d9f67e-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349137,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":109},\"maxValues\":{\"id\":109},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000012.json b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000012.json new file mode 100644 index 0000000000..4cc44fa8e8 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000012.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317349950,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":11,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4818"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"9a035bdd-f892-4449-9c39-401f31fcada6"}} +{"add":{"path":"part-00001-f3b19100-b5b3-4e72-8658-7a937e9ed515-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349924,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":110},\"maxValues\":{\"id\":110},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-54f2324a-e97f-4def-9101-9cc10599ba06-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349919,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":111},\"maxValues\":{\"id\":111},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-3f7ca40a-6497-4208-8a1a-11062456a5a9-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349936,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":112},\"maxValues\":{\"id\":112},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-a14852b2-c743-4a4a-b9c1-0c9472c51699-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349929,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":113},\"maxValues\":{\"id\":113},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-97d06207-5584-43df-afc2-2d1738d79193-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349943,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":114},\"maxValues\":{\"id\":114},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-0d431f03-6dbf-40e7-96fc-b1ebbbe9fc65-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349922,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":115},\"maxValues\":{\"id\":115},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-af0f0232-33c8-4315-821b-8bb1323b7a26-c000.snappy.parquet","partitionValues":{},"size":451,"modificationTime":1742317349936,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":116},\"maxValues\":{\"id\":116},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-4f744428-d088-497e-afd3-0b374e453e7c-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349936,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":117},\"maxValues\":{\"id\":117},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-694064b8-137e-45cd-b2ea-e28af172a2dc-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349918,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":118},\"maxValues\":{\"id\":118},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-56073753-4c1c-4a68-9b4a-13ef5d1a75fb-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317349938,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":119},\"maxValues\":{\"id\":119},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000013.json b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000013.json new file mode 100644 index 0000000000..b2d03d3ead --- /dev/null +++ b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000000013.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1742317350712,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":12,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"11","numOutputRows":"10","numOutputBytes":"4819"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"c3cd0fa1-9c72-4344-8225-0b787e52d5e0"}} +{"add":{"path":"part-00001-7a0d95f8-e122-4cf6-b89c-389036a9b415-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350696,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":120},\"maxValues\":{\"id\":120},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-f1f035c1-bf0f-485c-950d-c81d0d2aa8a2-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350706,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":121},\"maxValues\":{\"id\":121},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-d7a51e45-70f3-4379-819b-341951abefff-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350691,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":122},\"maxValues\":{\"id\":122},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-4828722c-5799-4be1-ace1-14bd7f477dbf-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350691,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":123},\"maxValues\":{\"id\":123},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-f3c3c72e-5d71-4dc9-9e15-342f1d6cb6cc-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350701,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":124},\"maxValues\":{\"id\":124},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-e8d74ede-8876-4f55-8e9f-1bbde0d07a35-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350696,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":125},\"maxValues\":{\"id\":125},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-91052146-2292-45c3-b57e-1fd2dd6be6ed-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350692,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":126},\"maxValues\":{\"id\":126},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-9d203964-9f4c-4c84-ad77-9ba305bb6572-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350706,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":127},\"maxValues\":{\"id\":127},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00010-8362228b-acf6-4937-875b-26c013c342e1-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350690,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":128},\"maxValues\":{\"id\":128},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00011-6f57658e-1953-4b59-b504-27c9e8c5cc3b-c000.snappy.parquet","partitionValues":{},"size":452,"modificationTime":1742317350677,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":129},\"maxValues\":{\"id\":129},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000009999.checkpoint.parquet b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000009999.checkpoint.parquet new file mode 100644 index 0000000000..4d9437c57f Binary files /dev/null and b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000009999.checkpoint.parquet differ diff --git a/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000009999.json b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000009999.json new file mode 100644 index 0000000000..37d128c02c --- /dev/null +++ b/crates/core/tests/data_err_logs/table_i/_delta_log/00000000000000009999.json @@ -0,0 +1,4 @@ +{"commitInfo":{"timestamp":1742317197774,"engineInfo":"Kernel-3.4.0-SNAPSHOT/test-engine","operation":"CREATE TABLE","operationParameters":{"partitionBy":"[]"},"isBlindAppend":true,"txnId":"09e63766-07ed-4b57-a6e0-0bba30aa801e","operationMetrics":{}}} +{"metaData":{"id":"7998c165-cb06-408d-bba0-e5c4e532ba21","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"createdTime":1742317196814,"configuration":{"delta.feature.catalogowned":"supported"}}} +{"protocol":{"minReaderVersion":3,"minWriterVersion":7,"readerFeatures":["catalogOwned"],"writerFeatures":["catalogOwned","appendOnly","invariants"]}} +{"add":{"path":"a735be79-daea-40b6-94f0-317a77a03df6-000.parquet","partitionValues":{},"size":1918,"modificationTime":1742317197691,"dataChange":true,"stats":"{\"numRecords\":600,\"minValues\":{\"id\":16},\"maxValues\":{\"id\":3251},\"nullCounts\":{\"id\":30}}"}} diff --git a/crates/core/tests/data_err_logs/table_i/_delta_log/_last_checkpoint b/crates/core/tests/data_err_logs/table_i/_delta_log/_last_checkpoint new file mode 100644 index 0000000000..604c2b31a2 --- /dev/null +++ b/crates/core/tests/data_err_logs/table_i/_delta_log/_last_checkpoint @@ -0,0 +1 @@ +{"version":10,"size":102,"sizeInBytes":21615,"numOfAddFiles":100,"checkpointSchema":{"type":"struct","fields":[{"name":"txn","type":{"type":"struct","fields":[{"name":"appId","type":"string","nullable":true,"metadata":{}},{"name":"version","type":"long","nullable":true,"metadata":{}},{"name":"lastUpdated","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"add","type":{"type":"struct","fields":[{"name":"path","type":"string","nullable":true,"metadata":{}},{"name":"partitionValues","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"size","type":"long","nullable":true,"metadata":{}},{"name":"modificationTime","type":"long","nullable":true,"metadata":{}},{"name":"dataChange","type":"boolean","nullable":true,"metadata":{}},{"name":"tags","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"deletionVector","type":{"type":"struct","fields":[{"name":"storageType","type":"string","nullable":true,"metadata":{}},{"name":"pathOrInlineDv","type":"string","nullable":true,"metadata":{}},{"name":"offset","type":"integer","nullable":true,"metadata":{}},{"name":"sizeInBytes","type":"integer","nullable":true,"metadata":{}},{"name":"cardinality","type":"long","nullable":true,"metadata":{}},{"name":"maxRowIndex","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"baseRowId","type":"long","nullable":true,"metadata":{}},{"name":"defaultRowCommitVersion","type":"long","nullable":true,"metadata":{}},{"name":"clusteringProvider","type":"string","nullable":true,"metadata":{}},{"name":"stats","type":"string","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"remove","type":{"type":"struct","fields":[{"name":"path","type":"string","nullable":true,"metadata":{}},{"name":"deletionTimestamp","type":"long","nullable":true,"metadata":{}},{"name":"dataChange","type":"boolean","nullable":true,"metadata":{}},{"name":"extendedFileMetadata","type":"boolean","nullable":true,"metadata":{}},{"name":"partitionValues","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"size","type":"long","nullable":true,"metadata":{}},{"name":"deletionVector","type":{"type":"struct","fields":[{"name":"storageType","type":"string","nullable":true,"metadata":{}},{"name":"pathOrInlineDv","type":"string","nullable":true,"metadata":{}},{"name":"offset","type":"integer","nullable":true,"metadata":{}},{"name":"sizeInBytes","type":"integer","nullable":true,"metadata":{}},{"name":"cardinality","type":"long","nullable":true,"metadata":{}},{"name":"maxRowIndex","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"baseRowId","type":"long","nullable":true,"metadata":{}},{"name":"defaultRowCommitVersion","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"metaData","type":{"type":"struct","fields":[{"name":"id","type":"string","nullable":true,"metadata":{}},{"name":"name","type":"string","nullable":true,"metadata":{}},{"name":"description","type":"string","nullable":true,"metadata":{}},{"name":"format","type":{"type":"struct","fields":[{"name":"provider","type":"string","nullable":true,"metadata":{}},{"name":"options","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"schemaString","type":"string","nullable":true,"metadata":{}},{"name":"partitionColumns","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"configuration","type":{"type":"map","keyType":"string","valueType":"string","valueContainsNull":true},"nullable":true,"metadata":{}},{"name":"createdTime","type":"long","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"protocol","type":{"type":"struct","fields":[{"name":"minReaderVersion","type":"integer","nullable":true,"metadata":{}},{"name":"minWriterVersion","type":"integer","nullable":true,"metadata":{}},{"name":"readerFeatures","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"writerFeatures","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}},{"name":"domainMetadata","type":{"type":"struct","fields":[{"name":"domain","type":"string","nullable":true,"metadata":{}},{"name":"configuration","type":"string","nullable":true,"metadata":{}},{"name":"removed","type":"boolean","nullable":true,"metadata":{}}]},"nullable":true,"metadata":{}}]},"checksum":"94a578f92841fa7ba9cdee96b5905fdb"} diff --git a/crates/core/tests/eager_snapshot.rs b/crates/core/tests/eager_snapshot.rs new file mode 100644 index 0000000000..cdcf09776a --- /dev/null +++ b/crates/core/tests/eager_snapshot.rs @@ -0,0 +1,60 @@ +use std::collections::HashMap; + +use deltalake_core::{ + kernel::{transaction::CommitData, EagerSnapshot}, + protocol::{DeltaOperation, SaveMode}, + test_utils::add_as_remove, +}; +use deltalake_test::utils::*; +use itertools::Itertools; + +#[tokio::test] +async fn test_eager_snapshot_advance() -> TestResult { + let context = IntegrationContext::new(Box::::default())?; + context.load_table(TestTables::Simple).await?; + + let log_store = context.table_builder(TestTables::Simple).build_storage()?; + + let mut snapshot = EagerSnapshot::try_new(&log_store, Default::default(), None).await?; + + let version = snapshot.version(); + + let files = snapshot.file_actions()?.enumerate().collect_vec(); + let num_files = files.len(); + + let split = files.split(|(idx, _)| *idx == num_files / 2).collect_vec(); + assert!(split.len() == 2 && !split[0].is_empty() && !split[1].is_empty()); + let (first, second) = split.into_iter().next_tuple().unwrap(); + + let removes = first + .iter() + .map(|(_, add)| add_as_remove(add, add.data_change).into()) + .collect_vec(); + + let operation = DeltaOperation::Write { + mode: SaveMode::Append, + partition_by: None, + predicate: None, + }; + + let actions = vec![CommitData::new( + removes, + operation, + HashMap::new(), + Vec::new(), + )]; + + let new_version = snapshot.advance(&actions)?; + assert_eq!(new_version, version + 1); + + let new_files = snapshot.file_actions()?.map(|f| f.path).collect::>(); + assert_eq!(new_files.len(), num_files - first.len()); + assert!(first + .iter() + .all(|(_, add)| { !new_files.contains(&add.path) })); + assert!(second + .iter() + .all(|(_, add)| { new_files.contains(&add.path) })); + + Ok(()) +} diff --git a/crates/core/tests/exotic_tables.rs b/crates/core/tests/exotic_tables.rs new file mode 100644 index 0000000000..b9b1ea6073 --- /dev/null +++ b/crates/core/tests/exotic_tables.rs @@ -0,0 +1,28 @@ +#[tokio::test] +async fn test_exotic_tables() { + let dir = env!("CARGO_MANIFEST_DIR"); + let data_path = std::path::Path::new(dir).join("tests/data_err_logs"); + let full = data_path.canonicalize().unwrap(); + + let cases = vec![ + ("table_a", false), + ("table_b", false), + ("table_c", true), + ("table_d", true), + ("table_e", true), + ("table_f", true), + // ("table_g", false), + ("table_h", true), + ("table_i", true), + ]; + + for (name, should_error) in cases { + let table_path = full.join(name); + let table = deltalake_core::open_table(&table_path.to_string_lossy()).await; + if should_error { + assert!(table.is_err()); + } else { + assert!(table.is_ok()); + } + } +} diff --git a/crates/core/tests/fs_common/mod.rs b/crates/core/tests/fs_common/mod.rs index 061c0d8c58..e9d4985c5e 100644 --- a/crates/core/tests/fs_common/mod.rs +++ b/crates/core/tests/fs_common/mod.rs @@ -1,11 +1,11 @@ use chrono::Utc; +use deltalake_core::kernel::transaction::CommitBuilder; use deltalake_core::kernel::{ Action, Add, DataType, PrimitiveType, Remove, StructField, StructType, }; +use deltalake_core::logstore::object_store::{GetResult, Result as ObjectStoreResult}; use deltalake_core::operations::create::CreateBuilder; -use deltalake_core::operations::transaction::CommitBuilder; use deltalake_core::protocol::{DeltaOperation, SaveMode}; -use deltalake_core::storage::{GetResult, ObjectStoreResult, StorageOptions}; use deltalake_core::DeltaTable; use object_store::path::Path as StorePath; use object_store::{ @@ -89,7 +89,6 @@ pub fn add(offset_millis: i64) -> Add { modification_time: Utc::now().timestamp_millis() - offset_millis, data_change: true, stats: None, - stats_parsed: None, tags: None, deletion_vector: None, base_row_id: None, @@ -147,12 +146,9 @@ impl std::fmt::Display for SlowStore { impl SlowStore { #[allow(dead_code)] - pub fn new( - location: Url, - _options: impl Into + Clone, - ) -> deltalake_core::DeltaResult { + pub fn new(location: Url) -> deltalake_core::DeltaResult { Ok(Self { - inner: deltalake_core::storage::store_for(&location, &StorageOptions::default())?, + inner: deltalake_core::logstore::store_for(&location, None::<(&str, &str)>)?, }) } } @@ -195,7 +191,7 @@ impl ObjectStore for SlowStore { async fn get_range( &self, location: &StorePath, - range: std::ops::Range, + range: std::ops::Range, ) -> ObjectStoreResult { self.inner.get_range(location, range).await } @@ -217,7 +213,7 @@ impl ObjectStore for SlowStore { fn list( &self, prefix: Option<&StorePath>, - ) -> futures::stream::BoxStream<'_, ObjectStoreResult> { + ) -> futures::stream::BoxStream<'static, ObjectStoreResult> { self.inner.list(prefix) } @@ -229,7 +225,7 @@ impl ObjectStore for SlowStore { &self, prefix: Option<&StorePath>, offset: &StorePath, - ) -> futures::stream::BoxStream<'_, ObjectStoreResult> { + ) -> futures::stream::BoxStream<'static, ObjectStoreResult> { self.inner.list_with_offset(prefix, offset) } diff --git a/crates/core/tests/integration.rs b/crates/core/tests/integration.rs index 1dbd28f68a..3b37129682 100644 --- a/crates/core/tests/integration.rs +++ b/crates/core/tests/integration.rs @@ -66,7 +66,7 @@ async fn test_action_reconciliation() { table .snapshot() .unwrap() - .all_tombstones(table.object_store().clone()) + .all_tombstones(&table.log_store()) .await .unwrap() .map(|r| r.path.clone()) diff --git a/crates/core/tests/integration_checkpoint.rs b/crates/core/tests/integration_checkpoint.rs index ad542aef76..82c8bc6a3a 100644 --- a/crates/core/tests/integration_checkpoint.rs +++ b/crates/core/tests/integration_checkpoint.rs @@ -100,13 +100,13 @@ async fn test_issue_1420_cleanup_expired_logs_for() -> DeltaResult<()> { writer.write(vec![json!({"id": 2})]).await?; writer.flush_and_commit(&mut table).await?; // v2 - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); create_checkpoint(&table, None).await.unwrap(); // v2.checkpoint.parquet // Should delete v1 but not v2 or v2.checkpoint.parquet cleanup_expired_logs_for( - table.version(), + table.version().unwrap(), table.log_store().as_ref(), ts.timestamp_millis(), None, @@ -115,7 +115,8 @@ async fn test_issue_1420_cleanup_expired_logs_for() -> DeltaResult<()> { assert!( table - .object_store() + .log_store() + .object_store(None) .head(&Path::from(format!("_delta_log/{:020}.json", 1))) .await .is_err(), @@ -124,7 +125,8 @@ async fn test_issue_1420_cleanup_expired_logs_for() -> DeltaResult<()> { assert!( table - .object_store() + .log_store() + .object_store(None) .head(&Path::from(format!("_delta_log/{:020}.json", 2))) .await .is_ok(), @@ -133,7 +135,8 @@ async fn test_issue_1420_cleanup_expired_logs_for() -> DeltaResult<()> { assert!( table - .object_store() + .log_store() + .object_store(None) .head(&Path::from(format!( "_delta_log/{:020}.checkpoint.parquet", 2 @@ -149,7 +152,7 @@ async fn test_issue_1420_cleanup_expired_logs_for() -> DeltaResult<()> { sleep(Duration::from_secs(1)).await; cleanup_expired_logs_for( - table.version(), + table.version().unwrap(), table.log_store().as_ref(), ts.timestamp_millis(), None, @@ -158,7 +161,8 @@ async fn test_issue_1420_cleanup_expired_logs_for() -> DeltaResult<()> { assert!( table - .object_store() + .log_store() + .object_store(None) .head(&Path::from(format!("_delta_log/{:020}.json", 2))) .await .is_ok(), @@ -167,7 +171,8 @@ async fn test_issue_1420_cleanup_expired_logs_for() -> DeltaResult<()> { assert!( table - .object_store() + .log_store() + .object_store(None) .head(&Path::from(format!( "_delta_log/{:020}.checkpoint.parquet", 2 diff --git a/crates/core/tests/integration_datafusion.rs b/crates/core/tests/integration_datafusion.rs index 990339b942..44f154807a 100644 --- a/crates/core/tests/integration_datafusion.rs +++ b/crates/core/tests/integration_datafusion.rs @@ -13,6 +13,7 @@ use arrow_schema::{ use datafusion::assert_batches_sorted_eq; use datafusion::datasource::TableProvider; use datafusion::execution::context::{SessionContext, SessionState, TaskContext}; +use datafusion::execution::SessionStateBuilder; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::physical_plan::{common::collect, metrics::Label}; use datafusion::physical_plan::{visit_execution_plan, ExecutionPlan, ExecutionPlanVisitor}; @@ -23,9 +24,10 @@ use datafusion_expr::Expr; use datafusion_proto::bytes::{ logical_plan_from_bytes_with_extension_codec, logical_plan_to_bytes_with_extension_codec, }; -use deltalake_core::delta_datafusion::DeltaScan; +use deltalake_core::delta_datafusion::{DeltaScan, DeltaTableFactory}; use deltalake_core::kernel::{DataType, MapType, PrimitiveType, StructField, StructType}; use deltalake_core::operations::create::CreateBuilder; +use deltalake_core::operations::write::SchemaMode; use deltalake_core::protocol::SaveMode; use deltalake_core::writer::{DeltaWriter, RecordBatchWriter}; use deltalake_core::{ @@ -33,22 +35,36 @@ use deltalake_core::{ operations::{write::WriteBuilder, DeltaOps}, DeltaTable, DeltaTableError, }; -use deltalake_test::datafusion::*; use deltalake_test::utils::*; use serial_test::serial; use url::Url; +pub fn context_with_delta_table_factory() -> SessionContext { + let mut state = SessionStateBuilder::new().build(); + state + .table_factories_mut() + .insert("DELTATABLE".to_string(), Arc::new(DeltaTableFactory {})); + SessionContext::new_with_state(state) +} + mod local { + use super::*; use datafusion::datasource::source::DataSourceExec; + use datafusion::prelude::SessionConfig; use datafusion::{common::stats::Precision, datasource::provider_as_source}; - use datafusion_expr::LogicalPlanBuilder; + use datafusion_common::assert_contains; + use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion}; + use datafusion_expr::{ + LogicalPlan, LogicalPlanBuilder, TableProviderFilterPushDown, TableScan, + }; + use datafusion_physical_plan::displayable; + use delta_kernel::engine::arrow_conversion::TryIntoKernel as _; use deltalake_core::{ delta_datafusion::DeltaLogicalCodec, logstore::default_logstore, writer::JsonWriter, }; use itertools::Itertools; use object_store::local::LocalFileSystem; - - use super::*; + use TableProviderFilterPushDown::{Exact, Inexact}; #[tokio::test] #[serial] async fn test_datafusion_local() -> TestResult { @@ -110,7 +126,7 @@ mod local { let table_dir = tempfile::tempdir().unwrap(); let table_path = table_dir.path(); let table_uri = table_path.to_str().unwrap().to_string(); - let table_schema: StructType = batches[0].schema().try_into().unwrap(); + let table_schema: StructType = batches[0].schema().try_into_kernel().unwrap(); let mut table = DeltaOps::try_from_uri(table_uri) .await @@ -188,6 +204,157 @@ mod local { Ok(()) } + struct PruningTestCase { + sql: String, + push_down: Vec, + } + + impl PruningTestCase { + fn new(sql: &str) -> Self { + Self { + sql: sql.to_string(), + push_down: vec![Exact], + } + } + + fn with_push_down(sql: &str, push_down: Vec) -> Self { + Self { + sql: sql.to_string(), + push_down, + } + } + } + + #[tokio::test] + async fn test_datafusion_optimize_stats_partitioned_pushdown() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(2); + let ctx = SessionContext::new_with_config(config); + let table = open_table("../test/tests/data/http_requests").await?; + ctx.register_table("http_requests", Arc::new(table.clone()))?; + + let sql = "SELECT COUNT(*) as num_events FROM http_requests WHERE date > '2023-04-13'"; + let df = ctx.sql(sql).await?; + let plan = df.clone().create_physical_plan().await?; + + // convert to explain plan form + let display = displayable(plan.as_ref()).indent(true).to_string(); + + assert_contains!( + &display, + "ProjectionExec: expr=[1437 as num_events]\n PlaceholderRowExec" + ); + + let batches = df.collect().await?; + let batch = &batches[0]; + + assert_eq!( + batch.column(0).as_ref(), + Arc::new(Int64Array::from(vec![1437])).as_ref(), + ); + + Ok(()) + } + + #[tokio::test] + async fn test_datafusion_query_partitioned_pushdown() -> Result<()> { + let ctx = SessionContext::new(); + let table = open_table("../test/tests/data/delta-0.8.0-partitioned").await?; + ctx.register_table("demo", Arc::new(table.clone()))?; + + let pruning_predicates = [ + PruningTestCase::new("year > '2020'"), + PruningTestCase::new("year != '2020'"), + PruningTestCase::new("year = '2021'"), + PruningTestCase::with_push_down( + "year = '2021' AND day IS NOT NULL", + vec![Exact, Exact], + ), + PruningTestCase::new("year IN ('2021', '2022')"), + // NOT IN (a, b) is rewritten as (col != a AND col != b) + PruningTestCase::with_push_down("year NOT IN ('2020', '2022')", vec![Exact, Exact]), + // BETWEEN a AND b is rewritten as (col >= a AND col < b) + PruningTestCase::with_push_down("year BETWEEN '2021' AND '2022'", vec![Exact, Exact]), + PruningTestCase::new("year NOT BETWEEN '2019' AND '2020'"), + PruningTestCase::with_push_down( + "year = '2021' AND day IN ('4', '5', '20')", + vec![Exact, Exact], + ), + PruningTestCase::with_push_down( + "year = '2021' AND cast(day as int) <= 20", + vec![Exact, Inexact], + ), + ]; + + fn find_scan_filters(plan: &LogicalPlan) -> Vec<&Expr> { + let mut result = vec![]; + + plan.apply(|node| { + if let LogicalPlan::TableScan(TableScan { ref filters, .. }) = node { + result = filters.iter().collect(); + Ok(TreeNodeRecursion::Stop) // Stop traversal once found + } else { + Ok(TreeNodeRecursion::Continue) // Continue traversal + } + }) + .expect("Traversal should not fail"); + + result + } + + for pp in pruning_predicates { + let pred = pp.sql; + let sql = format!("SELECT CAST( day as int ) as my_day FROM demo WHERE {pred} ORDER BY CAST( day as int ) ASC"); + println!("\nExecuting query: {}", sql); + + let df = ctx.sql(sql.as_str()).await?; + + // validate that we are correctly qualifying filters as Exact or Inexact + let plan = df.clone().into_optimized_plan()?; + let filters = find_scan_filters(&plan); + let push_down = table.supports_filters_pushdown(&filters)?; + + assert_eq!(push_down, pp.push_down); + + let batches = df.collect().await?; + + let batch = &batches[0]; + + assert_eq!( + batch.column(0).as_ref(), + Arc::new(Int32Array::from(vec![4, 5, 20, 20])).as_ref(), + ); + } + + Ok(()) + } + + #[tokio::test] + async fn test_files_scanned_pushdown_limit() -> Result<()> { + use datafusion::prelude::*; + let ctx = SessionContext::new(); + let state = ctx.state(); + let table = open_table("../test/tests/data/delta-0.8.0").await?; + + // Simple Equality test, we only exercise the limit in this test + let e = col("value").eq(lit(2)); + let metrics = get_scan_metrics(&table, &state, &[e.clone()]).await?; + assert_eq!(metrics.num_scanned_files(), 2); + assert_eq!(metrics.num_scanned_files(), metrics.keep_count); + assert_eq!(metrics.skip_count, 0); + + let metrics = get_scan_metrics_with_limit(&table, &state, &[e.clone()], Some(1)).await?; + assert_eq!(metrics.num_scanned_files(), 1); + assert_eq!(metrics.num_scanned_files(), metrics.keep_count); + assert_eq!(metrics.skip_count, 1); + + let metrics = get_scan_metrics_with_limit(&table, &state, &[e.clone()], Some(3)).await?; + assert_eq!(metrics.num_scanned_files(), 2); + assert_eq!(metrics.num_scanned_files(), metrics.keep_count); + assert_eq!(metrics.skip_count, 0); + + Ok(()) + } + #[tokio::test] async fn test_datafusion_write_from_serialized_delta_scan() -> Result<()> { // Build an execution plan for scanning a DeltaTable and serialize it to bytes. @@ -211,7 +378,7 @@ mod local { &ctx, &DeltaLogicalCodec {}, )?); - let schema = StructType::try_from(source_scan.schema().as_arrow()).unwrap(); + let schema: StructType = source_scan.schema().as_arrow().try_into_kernel().unwrap(); let fields = schema.fields().cloned(); dbg!(schema.fields().collect_vec().clone()); @@ -274,7 +441,7 @@ mod local { #[tokio::test] async fn test_datafusion_stats() -> Result<()> { - // Validate a table that contains statisitics for all files + // Validate a table that contains statistics for all files let table = open_table("../test/tests/data/delta-0.8.0").await.unwrap(); let statistics = table.snapshot()?.datafusion_table_statistics().unwrap(); @@ -312,7 +479,7 @@ mod local { ]; assert_batches_sorted_eq!(&expected, &actual); - // Validate a table that does not contain column statisitics + // Validate a table that does not contain column statistics let table = open_table("../test/tests/data/delta-0.2.0").await.unwrap(); let statistics = table.snapshot()?.datafusion_table_statistics().unwrap(); @@ -416,13 +583,14 @@ mod local { Ok(()) } - async fn get_scan_metrics( + async fn get_scan_metrics_with_limit( table: &DeltaTable, state: &SessionState, e: &[Expr], + limit: Option, ) -> Result { let mut metrics = ExecutionMetricsCollector::default(); - let scan = table.scan(state, None, e, None).await?; + let scan = table.scan(state, None, e, limit).await?; if scan.properties().output_partitioning().partition_count() > 0 { let plan = CoalescePartitionsExec::new(scan); let task_ctx = Arc::new(TaskContext::from(state)); @@ -437,6 +605,14 @@ mod local { Ok(metrics) } + async fn get_scan_metrics( + table: &DeltaTable, + state: &SessionState, + e: &[Expr], + ) -> Result { + get_scan_metrics_with_limit(table, state, e, None).await + } + fn create_all_types_batch( not_null_rows: usize, null_rows: usize, @@ -1084,13 +1260,14 @@ mod local { #[tokio::test] async fn test_issue_2105() -> Result<()> { use datafusion::arrow::datatypes::{DataType as ArrowDataType, Field, Schema}; - let path = tempfile::tempdir().unwrap(); - let path = path.into_path(); + let tmp_dir = tempfile::tempdir().unwrap(); + let path = tmp_dir.path(); - let file_store = LocalFileSystem::new_with_prefix(path.clone()).unwrap(); + let file_store = LocalFileSystem::new_with_prefix(path).unwrap(); let log_store = default_logstore( Arc::new(file_store), - &Url::from_file_path(path.clone()).unwrap(), + Arc::new(LocalFileSystem::new()), + &Url::from_file_path(path).unwrap(), &Default::default(), ); @@ -1182,6 +1359,79 @@ async fn simple_query(context: &IntegrationContext) -> TestResult { Ok(()) } +#[tokio::test] +async fn test_schema_adapter_empty_batch() { + let ctx = SessionContext::new(); + let tmp_dir = tempfile::tempdir().unwrap(); + let table_uri = tmp_dir.path().to_str().to_owned().unwrap(); + + // Create table with a single column + let table = DeltaOps::try_from_uri(table_uri) + .await + .unwrap() + .create() + .with_column( + "a", + DataType::Primitive(PrimitiveType::Integer), + false, + None, + ) + .await + .unwrap(); + + // Write single column + let a_arr = Int32Array::from(vec![1, 2, 3]); + let table = DeltaOps(table) + .write(vec![RecordBatch::try_from_iter_with_nullable(vec![( + "a", + Arc::new(a_arr) as ArrayRef, + false, + )]) + .unwrap()]) + .await + .unwrap(); + + // Evolve schema by writing a batch with new nullable column + let a_arr = Int32Array::from(vec![4, 5, 6]); + let b_arr = Int32Array::from(vec![7, 8, 9]); + let table = DeltaOps(table) + .write(vec![RecordBatch::try_from_iter_with_nullable(vec![ + ("a", Arc::new(a_arr) as ArrayRef, false), + ("b", Arc::new(b_arr) as ArrayRef, true), + ]) + .unwrap()]) + .with_schema_mode(SchemaMode::Merge) + .await + .unwrap(); + + // Ensure we can project only the new column which does not exist in files from first write + let batches = ctx + .read_table(Arc::new(table)) + .unwrap() + .select_exprs(&["b"]) + .unwrap() + .collect() + .await + .unwrap(); + + assert_batches_sorted_eq!( + #[rustfmt::skip] + &[ + "+---+", + "| b |", + "+---+", + "| |", + "| |", + "| |", + "| 7 |", + "| 8 |", + "| 9 |", + "+---+", + ], + &batches + ); +} + mod date_partitions { use super::*; diff --git a/crates/core/tests/read_delta_log_test.rs b/crates/core/tests/read_delta_log_test.rs index a6d2dc8833..51c1beb4a1 100644 --- a/crates/core/tests/read_delta_log_test.rs +++ b/crates/core/tests/read_delta_log_test.rs @@ -1,12 +1,12 @@ use deltalake_core::{DeltaResult, DeltaTableBuilder}; use pretty_assertions::assert_eq; -use std::collections::HashMap; use std::time::SystemTime; #[allow(dead_code)] mod fs_common; #[tokio::test] +#[ignore] async fn test_log_buffering() { let n_commits = 10; let path = "../test/tests/data/simple_table_with_no_checkpoint"; @@ -22,13 +22,7 @@ async fn test_log_buffering() { let location = deltalake_core::table::builder::ensure_table_uri(path).unwrap(); // use storage that sleeps 10ms on every `get` - let store = std::sync::Arc::new( - fs_common::SlowStore::new( - location.clone(), - deltalake_core::storage::StorageOptions::from(HashMap::new()), - ) - .unwrap(), - ); + let store = std::sync::Arc::new(fs_common::SlowStore::new(location.clone()).unwrap()); let mut seq_version = 0; let t = SystemTime::now(); @@ -42,7 +36,7 @@ async fn test_log_buffering() { .await .expect("Failed to load table"); table_seq.update_incremental(None).await.unwrap(); - seq_version = table_seq.version(); + seq_version = table_seq.version().unwrap(); } let time_seq = t.elapsed().unwrap(); @@ -58,7 +52,7 @@ async fn test_log_buffering() { .await .unwrap(); table_buf.update_incremental(None).await.unwrap(); - buf_version = table_buf.version(); + buf_version = table_buf.version().unwrap(); } let time_buf = t2.elapsed().unwrap(); @@ -89,7 +83,7 @@ async fn test_log_buffering_success_explicit_version() { .await .unwrap(); table.update_incremental(None).await.unwrap(); - assert_eq!(table.version(), 10); + assert_eq!(table.version(), Some(10)); let mut table = DeltaTableBuilder::from_uri(path) .with_version(0) @@ -99,7 +93,7 @@ async fn test_log_buffering_success_explicit_version() { .await .unwrap(); table.update_incremental(Some(0)).await.unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let mut table = DeltaTableBuilder::from_uri(path) .with_version(0) @@ -109,7 +103,7 @@ async fn test_log_buffering_success_explicit_version() { .await .unwrap(); table.update_incremental(Some(1)).await.unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let mut table = DeltaTableBuilder::from_uri(path) .with_version(0) @@ -119,7 +113,7 @@ async fn test_log_buffering_success_explicit_version() { .await .unwrap(); table.update_incremental(Some(10)).await.unwrap(); - assert_eq!(table.version(), 10); + assert_eq!(table.version(), Some(10)); let mut table = DeltaTableBuilder::from_uri(path) .with_version(0) @@ -129,7 +123,7 @@ async fn test_log_buffering_success_explicit_version() { .await .unwrap(); table.update_incremental(Some(20)).await.unwrap(); - assert_eq!(table.version(), 10); + assert_eq!(table.version(), Some(10)); } } @@ -144,6 +138,7 @@ async fn test_log_buffering_fail() { } #[tokio::test] +#[ignore = "not implemented"] async fn test_read_liquid_table() -> DeltaResult<()> { let path = "../test/tests/data/table_with_liquid_clustering"; let _table = deltalake_core::open_table(&path).await?; @@ -151,6 +146,7 @@ async fn test_read_liquid_table() -> DeltaResult<()> { } #[tokio::test] +#[ignore = "not implemented"] async fn test_read_table_features() -> DeltaResult<()> { let mut _table = deltalake_core::open_table("../test/tests/data/simple_table_features").await?; let rf = _table.protocol()?.reader_features.clone(); @@ -165,11 +161,12 @@ async fn test_read_table_features() -> DeltaResult<()> { // test for: https://github.com/delta-io/delta-rs/issues/1302 #[tokio::test] +#[ignore = "not implemented"] async fn read_delta_table_from_dlt() { let table = deltalake_core::open_table("../test/tests/data/delta-live-table") .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert!(table.get_schema().is_ok()); } @@ -179,15 +176,16 @@ async fn read_delta_table_with_null_stats_in_notnull_struct() { deltalake_core::open_table("../test/tests/data/table_with_null_stats_in_notnull_struct") .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); assert!(table.get_schema().is_ok()); } #[tokio::test] +#[ignore = "not implemented"] async fn read_delta_table_with_renamed_partitioning_column() { let table = deltalake_core::open_table("../test/tests/data/table_with_partitioning_mapping") .await .unwrap(); - assert_eq!(table.version(), 4); + assert_eq!(table.version(), Some(4)); assert!(table.get_schema().is_ok()); } diff --git a/crates/core/tests/read_delta_partitions_test.rs b/crates/core/tests/read_delta_partitions_test.rs index 3ccc120ae4..0e6ae5c4d4 100644 --- a/crates/core/tests/read_delta_partitions_test.rs +++ b/crates/core/tests/read_delta_partitions_test.rs @@ -48,7 +48,7 @@ async fn read_null_partitions_from_checkpoint() { let table = deltalake_core::open_table(&table.table_uri()) .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); } #[cfg(feature = "datafusion")] diff --git a/crates/core/tests/time_travel.rs b/crates/core/tests/time_travel.rs index 3277a41961..839f2ed4b2 100644 --- a/crates/core/tests/time_travel.rs +++ b/crates/core/tests/time_travel.rs @@ -5,6 +5,8 @@ use std::time::SystemTime; #[tokio::test] async fn time_travel_by_ds() { + // test time travel on a table with an uncommitted delta in a .tmp subfolder + // git does not preserve mtime, so we need to manually set it in the test let log_dir = "../test/tests/data/simple_table/_delta_log"; let log_mtime_pair = vec![ @@ -13,6 +15,11 @@ async fn time_travel_by_ds() { ("00000000000000000002.json", "2020-05-03T22:47:31-07:00"), ("00000000000000000003.json", "2020-05-04T22:47:31-07:00"), ("00000000000000000004.json", "2020-05-05T22:47:31-07:00"), + // Final file is uncommitted by Spark and is in a .tmp subdir + ( + ".tmp/00000000000000000005.json", + "2020-05-06T22:47:31-07:00", + ), ]; for (fname, ds) in log_mtime_pair { let ts: SystemTime = ds_to_ts(ds).into(); @@ -29,7 +36,7 @@ async fn time_travel_by_ds() { .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); table = deltalake_core::open_table_with_ds( "../test/tests/data/simple_table", @@ -37,7 +44,7 @@ async fn time_travel_by_ds() { ) .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); table = deltalake_core::open_table_with_ds( "../test/tests/data/simple_table", @@ -45,7 +52,7 @@ async fn time_travel_by_ds() { ) .await .unwrap(); - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); table = deltalake_core::open_table_with_ds( "../test/tests/data/simple_table", @@ -53,7 +60,7 @@ async fn time_travel_by_ds() { ) .await .unwrap(); - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); table = deltalake_core::open_table_with_ds( "../test/tests/data/simple_table", @@ -61,7 +68,7 @@ async fn time_travel_by_ds() { ) .await .unwrap(); - assert_eq!(table.version(), 3); + assert_eq!(table.version(), Some(3)); table = deltalake_core::open_table_with_ds( "../test/tests/data/simple_table", @@ -69,7 +76,7 @@ async fn time_travel_by_ds() { ) .await .unwrap(); - assert_eq!(table.version(), 3); + assert_eq!(table.version(), Some(3)); table = deltalake_core::open_table_with_ds( "../test/tests/data/simple_table", @@ -77,15 +84,16 @@ async fn time_travel_by_ds() { ) .await .unwrap(); - assert_eq!(table.version(), 4); + assert_eq!(table.version(), Some(4)); + // Final append in .tmp subdir is uncommitted and should be ignored table = deltalake_core::open_table_with_ds( "../test/tests/data/simple_table", "2020-05-25T22:47:31-07:00", ) .await .unwrap(); - assert_eq!(table.version(), 4); + assert_eq!(table.version(), Some(4)); } fn ds_to_ts(ds: &str) -> DateTime { diff --git a/crates/deltalake/Cargo.toml b/crates/deltalake/Cargo.toml index f6178585fb..755ca7b249 100644 --- a/crates/deltalake/Cargo.toml +++ b/crates/deltalake/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "deltalake" -version = "0.25.0" +version = "0.26.2" authors.workspace = true keywords.workspace = true readme.workspace = true @@ -13,17 +13,27 @@ rust-version.workspace = true [package.metadata.docs.rs] # We cannot use all_features because TLS features are mutually exclusive. -features = ["azure", "datafusion", "gcs", "hdfs", "json", "python", "s3", "unity-experimental"] +features = [ + "azure", + "datafusion", + "gcs", + "hdfs", + "json", + "python", + "s3", + "unity-experimental", +] [dependencies] -deltalake-core = { version = "0.25.0", path = "../core" } -deltalake-aws = { version = "0.8.0", path = "../aws", default-features = false, optional = true } -deltalake-azure = { version = "0.8.0", path = "../azure", optional = true } -deltalake-gcp = { version = "0.9.0", path = "../gcp", optional = true } -deltalake-hdfs = { version = "0.9.0", path = "../hdfs", optional = true } -deltalake-lakefs = { version = "0.8.0", path = "../lakefs", optional = true } -deltalake-catalog-glue = { version = "0.9.0", path = "../catalog-glue", optional = true } -deltalake-catalog-unity = { version = "0.9.0", path = "../catalog-unity", optional = true } +deltalake-core = { version = "0.26.0", path = "../core", default-features = false } +deltalake-aws = { version = "0.9.0", path = "../aws", default-features = false, optional = true } +deltalake-azure = { version = "0.9.0", path = "../azure", optional = true } +deltalake-gcp = { version = "0.10.0", path = "../gcp", optional = true } +deltalake-hdfs = { version = "0.10.0", path = "../hdfs", optional = true } +deltalake-lakefs = { version = "0.9.0", path = "../lakefs", optional = true } +deltalake-catalog-glue = { version = "0.10.0", path = "../catalog-glue", optional = true } +deltalake-catalog-unity = { version = "0.10.0", path = "../catalog-unity", optional = true } +delta_kernel = { workspace = true } [features] diff --git a/crates/deltalake/examples/basic_operations.rs b/crates/deltalake/examples/basic_operations.rs index 683588b782..9140e8cc7a 100644 --- a/crates/deltalake/examples/basic_operations.rs +++ b/crates/deltalake/examples/basic_operations.rs @@ -81,7 +81,7 @@ async fn main() -> Result<(), deltalake::errors::DeltaTableError> { .with_comment("A table to show how delta-rs works") .await?; - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); let writer_properties = WriterProperties::builder() .set_compression(Compression::ZSTD(ZstdLevel::try_new(3).unwrap())) @@ -93,7 +93,7 @@ async fn main() -> Result<(), deltalake::errors::DeltaTableError> { .with_writer_properties(writer_properties) .await?; - assert_eq!(table.version(), 1); + assert_eq!(table.version(), Some(1)); let writer_properties = WriterProperties::builder() .set_compression(Compression::ZSTD(ZstdLevel::try_new(3).unwrap())) @@ -106,7 +106,7 @@ async fn main() -> Result<(), deltalake::errors::DeltaTableError> { .with_writer_properties(writer_properties) .await?; - assert_eq!(table.version(), 2); + assert_eq!(table.version(), Some(2)); let (_table, stream) = DeltaOps(table).load().await?; let data: Vec = collect_sendable_stream(stream).await?; diff --git a/crates/deltalake/examples/recordbatch-writer.rs b/crates/deltalake/examples/recordbatch-writer.rs index 2e7b6da862..b8ef50eac1 100644 --- a/crates/deltalake/examples/recordbatch-writer.rs +++ b/crates/deltalake/examples/recordbatch-writer.rs @@ -7,10 +7,11 @@ * */ use chrono::prelude::*; +use delta_kernel::engine::arrow_conversion::TryIntoArrow as _; use deltalake::arrow::array::*; use deltalake::arrow::record_batch::RecordBatch; use deltalake::errors::DeltaTableError; -use deltalake::kernel::{DataType, PrimitiveType, StructField, StructType}; +use deltalake::kernel::{DataType, PrimitiveType, StructField}; use deltalake::parquet::{ basic::{Compression, ZstdLevel}, file::properties::WriterProperties, @@ -161,10 +162,10 @@ fn convert_to_batch(table: &DeltaTable, records: &Vec) -> RecordB let metadata = table .metadata() .expect("Failed to get metadata for the table"); - let arrow_schema = >::try_from( - &metadata.schema().expect("failed to get schema"), - ) - .expect("Failed to convert to arrow schema"); + let arrow_schema: deltalake::arrow::datatypes::Schema = + (&(metadata.schema().expect("failed to get schema"))) + .try_into_arrow() + .expect("Failed to convert to arrow schema"); let arrow_schema_ref = Arc::new(arrow_schema); let mut ts = vec![]; diff --git a/crates/deltalake/src/lib.rs b/crates/deltalake/src/lib.rs index a2887f2146..5bdd7a1ee3 100644 --- a/crates/deltalake/src/lib.rs +++ b/crates/deltalake/src/lib.rs @@ -3,6 +3,9 @@ */ pub use deltalake_core::*; +#[cfg(not(any(feature = "rustls", feature = "native-tls")))] +compile_error!("You must enable at least one of the features: `rustls` or `native-tls`."); + #[cfg(any(feature = "s3", feature = "s3-native-tls"))] pub use deltalake_aws as aws; #[cfg(feature = "azure")] diff --git a/crates/derive/Cargo.toml b/crates/derive/Cargo.toml new file mode 100644 index 0000000000..1b6cff882b --- /dev/null +++ b/crates/derive/Cargo.toml @@ -0,0 +1,23 @@ +[package] +name = "deltalake-derive" +version = "0.26.0" +description = "Dervice macros for use in delta ecosystem crates" +authors.workspace = true +rust-version.workspace = true +keywords.workspace = true +readme.workspace = true +edition.workspace = true +homepage.workspace = true +license.workspace = true +documentation.workspace = true +repository.workspace = true + +[lib] +proc-macro = true + +[dependencies] +proc-macro2 = "1" +syn = { version = "2.0", features = ["extra-traits"] } +quote = "1.0" +convert_case = "0.8.0" +itertools = "0.14" diff --git a/crates/derive/LICENSE.txt b/crates/derive/LICENSE.txt new file mode 120000 index 0000000000..1ef648f64b --- /dev/null +++ b/crates/derive/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/crates/derive/src/lib.rs b/crates/derive/src/lib.rs new file mode 100644 index 0000000000..3c5b38af53 --- /dev/null +++ b/crates/derive/src/lib.rs @@ -0,0 +1,356 @@ +use convert_case::{Case, Casing}; +use itertools::Itertools; +use proc_macro::TokenStream; +use proc_macro2::Span; +use quote::quote; +use syn::parse::Parser; +use syn::{ + parse_macro_input, punctuated::Punctuated, Data, DeriveInput, Field, Fields, Lit, Meta, + MetaNameValue, Token, Type, +}; +use syn::{Attribute, Error, Expr, Ident, Result}; + +/// Derive macro for implementing the TryUpdateKey trait +/// +/// This macro automatically implements TryUpdateKey for a struct, +/// mapping field names to configuration keys and using appropriate parsers +/// based on the field type. +/// +/// Additional key aliases can be specified with the `#[delta(alias = "alias.name")]` attribute. +/// Multiple aliases can be added by `#[delta(alias = "foo", alias = "bar")]`. +/// +/// Reading configuration can be achieved by assigning environmene keys to a field +/// `#[delta(env = "MY_ENV_KEY")]`. +#[proc_macro_derive(DeltaConfig, attributes(delta))] +pub fn derive_delta_config(input: TokenStream) -> TokenStream { + // Parse the input tokens into a syntax tree + let input = parse_macro_input!(input as DeriveInput); + + // Get the name of the struct + let name = &input.ident; + + // Extract the fields from the struct + let fields = match &input.data { + Data::Struct(data) => match &data.fields { + Fields::Named(fields) => &fields.named, + _ => panic!("TryUpdateKey can only be derived for structs with named fields"), + }, + _ => panic!("TryUpdateKey can only be derived for structs"), + } + .into_iter() + .collect::>(); + + // Generate the implementation for TryUpdateKey trait + let try_update_key = match generate_try_update_key(name, &fields) { + Ok(try_update_key) => try_update_key, + Err(err) => return syn::Error::into_compile_error(err).into(), + }; + + // generate an enum with all configuration keys + let config_keys = match generate_config_keys(name, &fields) { + Ok(config_keys) => config_keys, + Err(err) => return syn::Error::into_compile_error(err).into(), + }; + + // generate a FromIterator implementation + let from_iter = generate_from_iterator(name); + + let expanded = quote! { + #try_update_key + + #config_keys + + #from_iter + }; + + TokenStream::from(expanded) +} + +fn generate_config_keys(name: &Ident, fields: &[&Field]) -> Result { + let enum_name = Ident::new(&format!("{}Key", name), Span::call_site()); + let variants: Vec<_> = fields + .iter() + .map(|field| { + let field_name = &field + .ident + .as_ref() + .ok_or_else(|| syn::Error::new_spanned(field, "expected name"))? + .to_string(); + let pascal_case = Ident::new(&field_name.to_case(Case::Pascal), Span::call_site()); + let attributes = extract_field_attributes(&field.attrs)?; + + // Generate doc attribute if documentation exists + let doc_attr = if let Some(doc_string) = attributes.docs { + // Create a doc attribute for the enum variant + quote! { #[doc = #doc_string] } + } else { + // No documentation + quote! {} + }; + + // Return the variant with its documentation + Ok(quote! { + #doc_attr + #pascal_case + }) + }) + .collect::>()?; + Ok(quote! { + #[automatically_derived] + pub enum #enum_name { + #(#variants),* + } + }) +} + +fn generate_from_iterator(name: &Ident) -> proc_macro2::TokenStream { + quote! { + #[automatically_derived] + impl FromIterator<(K, V)> for #name + where + K: AsRef + Into, + V: AsRef + Into, + { + fn from_iter>(iter: I) -> Self { + crate::logstore::config::ParseResult::from_iter(iter).config + } + } + } +} + +fn generate_try_update_key(name: &Ident, fields: &[&Field]) -> Result { + let match_arms: Vec<_> = fields + .iter() + .filter_map(|field| { + let field_name = &field.ident.as_ref().unwrap(); + let field_name_str = field_name.to_string(); + + // Extract aliases from attributes + let attributes = match extract_field_attributes(&field.attrs) { + Ok(attributes) => attributes, + Err(e) => return Some(Err(e)), + }; + if attributes.skip { + return None; + } + + // Determine parser based on field type + let (parser, is_option) = match determine_parser(&field.ty) { + Ok((parser, is_option)) => (parser, is_option), + Err(e) => return Some(Err(e)), + }; + + // Build the match conditions: field name and all aliases + let mut match_conditions = vec![quote! { #field_name_str }]; + for alias in attributes.aliases { + match_conditions.push(quote! { #alias }); + } + + let match_arm = if is_option { + quote! { + #(#match_conditions)|* => self.#field_name = Some(#parser(v)?), + } + } else { + quote! { + #(#match_conditions)|* => self.#field_name = #parser(v)?, + } + }; + + Some(Ok(match_arm)) + }) + .try_collect()?; + + let env_setters = generate_load_from_env(fields)?; + + Ok(quote! { + #[automatically_derived] + impl crate::logstore::config::TryUpdateKey for #name { + fn try_update_key(&mut self, key: &str, v: &str) -> crate::DeltaResult> { + match key { + #(#match_arms)* + _ => return Ok(None), + } + Ok(Some(())) + } + + fn load_from_environment(&mut self) -> crate::DeltaResult<()> { + let default_values = Self::default(); + #(#env_setters)* + Ok(()) + } + } + }) +} + +fn generate_load_from_env(fields: &[&Field]) -> Result> { + fields.iter().filter_map(|field| { + let field_name = &field.ident.as_ref().unwrap(); + let attributes = match extract_field_attributes(&field.attrs) { + Ok(attributes) => attributes, + Err(e) => return Some(Err(e)), + }; + if attributes.skip || attributes.env_variable_names.is_empty() { + return None; + } + + let (parser, is_option) = match determine_parser(&field.ty) { + Ok((parser, is_option)) => (parser, is_option), + Err(e) => return Some(Err(e)) + }; + + let env_checks = attributes.env_variable_names.iter().map(|env_var| { + if is_option { + // For Option types, only set if None + quote! { + if self.#field_name.is_none() { + if let Ok(val) = std::env::var(#env_var) { + match #parser(&val) { + Ok(parsed) => self.#field_name = Some(parsed), + Err(e) => ::tracing::warn!("Failed to parse environment variable {}: {}", #env_var, e), + } + } + } + } + } else { + // For non-Option types, we override the default value + // but ignore it if the current value is not the default. + quote! { + if self.#field_name == default_values.#field_name { + if let Ok(val) = std::env::var(#env_var) { + match #parser(&val) { + Ok(parsed) => self.#field_name = parsed, + Err(e) => ::tracing::warn!("Failed to parse environment variable {}: {}", #env_var, e), + } + } + } + } + } + }); + + Some(Ok(quote! { + #(#env_checks)* + })) + }).try_collect() +} + +// Helper function to determine the appropriate parser based on field type +fn determine_parser(ty: &Type) -> Result<(proc_macro2::TokenStream, bool)> { + match ty { + Type::Path(type_path) => { + let type_str = quote! { #type_path }.to_string(); + let is_option = type_str.starts_with("Option"); + + let caller = if type_str.contains("usize") { + quote! { crate::logstore::config::parse_usize } + } else if type_str.contains("f64") || type_str.contains("f32") { + quote! { crate::logstore::config::parse_f64 } + } else if type_str.contains("Duration") { + quote! { crate::logstore::config::parse_duration } + } else if type_str.contains("bool") { + quote! { crate::logstore::config::parse_bool } + } else if type_str.contains("String") { + quote! { crate::logstore::config::parse_string } + } else { + return Err(Error::new_spanned(ty, + format!("Unsupported field type: {type_str}. Consider implementing a custom parser.") + )); + }; + + Ok((caller, is_option)) + } + _ => panic!("Unsupported field type for TryUpdateKey"), + } +} + +struct FieldAttributes { + aliases: Vec, + env_variable_names: Vec, + docs: Option, + skip: bool, +} + +/// Extract aliases from field attributes +/// +/// Parse the annotations from individual fields into a convenient structure. +/// The field is annotated via `#[delta(...)]`. The following attributes are supported: +/// - `alias = "alias_name"`: Specifies an alias for the field. +/// - `env = "env_name"`: Specifies an environment variable name for the field. +/// - `skip`: Specifies whether the field should be skipped during parsing. +fn extract_field_attributes(attrs: &[Attribute]) -> Result { + let mut aliases = Vec::new(); + let mut environments = Vec::new(); + let mut docs = None; + let mut doc_strings = Vec::new(); + let mut skip = false; + + for attr in attrs { + if attr.path().is_ident("doc") { + // Handle doc comments + let meta = attr.meta.require_name_value()?; + if let Expr::Lit(expr_lit) = &meta.value { + if let Lit::Str(lit_str) = &expr_lit.lit { + // Collect all doc strings - they might span multiple lines + doc_strings.push(lit_str.value().trim().to_string()); + } + } + } + if attr.path().is_ident("delta") { + match &attr.meta { + Meta::List(list) => { + let parser = Punctuated::::parse_terminated; + let parsed = parser.parse(list.tokens.clone().into())?; + + for val in parsed { + match val { + Meta::NameValue(MetaNameValue { path, value, .. }) => { + if path.is_ident("alias") { + if let Expr::Lit(lit_expr) = &value { + if let Lit::Str(lit_str) = &lit_expr.lit { + aliases.push(lit_str.value()); + } + } + } + if path.is_ident("environment") || path.is_ident("env") { + if let Expr::Lit(lit_expr) = &value { + if let Lit::Str(lit_str) = &lit_expr.lit { + environments.push(lit_str.value()); + } + } + } + } + Meta::Path(path) => { + if path.is_ident("skip") { + skip = true; + } + } + _ => { + return Err(Error::new_spanned( + &attr.meta, + "only NameValue and Path parameters are supported", + )); + } + } + } + } + _ => { + return Err(Error::new_spanned( + &attr.meta, + "expected a list-style attribute", + )); + } + } + } + } + + // Combine all doc strings into a single documentation string + if !doc_strings.is_empty() { + docs = Some(doc_strings.join("\n")); + } + + Ok(FieldAttributes { + aliases, + env_variable_names: environments, + docs, + skip, + }) +} diff --git a/crates/gcp/Cargo.toml b/crates/gcp/Cargo.toml index 55061fab9d..7a6ba06805 100644 --- a/crates/gcp/Cargo.toml +++ b/crates/gcp/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "deltalake-gcp" -version = "0.9.0" +version = "0.10.1" authors.workspace = true keywords.workspace = true readme.workspace = true @@ -12,7 +12,7 @@ repository.workspace = true rust-version.workspace = true [dependencies] -deltalake-core = { version = "0.25.0", path = "../core", features = ["cloud"] } +deltalake-core = { version = "0.26.0", path = "../core" } # workspace depenndecies async-trait = { workspace = true } diff --git a/crates/gcp/src/lib.rs b/crates/gcp/src/lib.rs index caef8b65a0..6523583005 100644 --- a/crates/gcp/src/lib.rs +++ b/crates/gcp/src/lib.rs @@ -2,14 +2,15 @@ use std::collections::HashMap; use std::str::FromStr; use std::sync::Arc; -use deltalake_core::logstore::{default_logstore, logstores, LogStore, LogStoreFactory}; -use deltalake_core::storage::{ - factories, limit_store_handler, url_prefix_handler, ObjectStoreFactory, ObjectStoreRef, - RetryConfigParse, StorageOptions, +use deltalake_core::logstore::object_store::gcp::{GoogleCloudStorageBuilder, GoogleConfigKey}; +use deltalake_core::logstore::object_store::{ObjectStoreScheme, RetryConfig}; +use deltalake_core::logstore::{default_logstore, logstore_factories, LogStore, LogStoreFactory}; +use deltalake_core::logstore::{ + object_store_factories, ObjectStoreFactory, ObjectStoreRef, StorageConfig, }; use deltalake_core::{DeltaResult, DeltaTableError, Path}; -use object_store::gcp::{GoogleCloudStorageBuilder, GoogleConfigKey}; -use object_store::ObjectStoreScheme; +use object_store::client::SpawnedReqwestConnector; +use tokio::runtime::Handle; use url::Url; mod config; @@ -20,10 +21,9 @@ trait GcpOptions { fn as_gcp_options(&self) -> HashMap; } -impl GcpOptions for StorageOptions { +impl GcpOptions for HashMap { fn as_gcp_options(&self) -> HashMap { - self.0 - .iter() + self.iter() .filter_map(|(key, value)| { Some(( GoogleConfigKey::from_str(&key.to_ascii_lowercase()).ok()?, @@ -37,13 +37,13 @@ impl GcpOptions for StorageOptions { #[derive(Clone, Default, Debug)] pub struct GcpFactory {} -impl RetryConfigParse for GcpFactory {} - impl ObjectStoreFactory for GcpFactory { fn parse_url_opts( &self, url: &Url, - options: &StorageOptions, + options: &HashMap, + retry: &RetryConfig, + handle: Option, ) -> DeltaResult<(ObjectStoreRef, Path)> { let config = config::GcpConfigHelper::try_new(options.as_gcp_options())?.build()?; @@ -55,28 +55,35 @@ impl ObjectStoreFactory for GcpFactory { let mut builder = GoogleCloudStorageBuilder::new().with_url(url.to_string()); + if let Some(handle) = handle { + builder = builder.with_http_connector(SpawnedReqwestConnector::new(handle)); + } + for (key, value) in config.iter() { builder = builder.with_config(*key, value.clone()); } - let inner = builder - .with_retry(self.parse_retry_config(options)?) - .build()?; + let inner = builder.with_retry(retry.clone()).build()?; + let store = crate::storage::GcsStorageBackend::try_new(Arc::new(inner))?; - let gcs_backend = crate::storage::GcsStorageBackend::try_new(Arc::new(inner))?; - let store = limit_store_handler(url_prefix_handler(gcs_backend, prefix.clone()), options); - Ok((store, prefix)) + Ok((Arc::new(store), prefix)) } } impl LogStoreFactory for GcpFactory { fn with_options( &self, - store: ObjectStoreRef, + prefixed_store: ObjectStoreRef, + root_store: ObjectStoreRef, location: &Url, - options: &StorageOptions, + options: &StorageConfig, ) -> DeltaResult> { - Ok(default_logstore(store, location, options)) + Ok(default_logstore( + prefixed_store, + root_store, + location, + options, + )) } } @@ -85,6 +92,6 @@ pub fn register_handlers(_additional_prefixes: Option) { let factory = Arc::new(GcpFactory {}); let scheme = &"gs"; let url = Url::parse(&format!("{scheme}://")).unwrap(); - factories().insert(url.clone(), factory.clone()); - logstores().insert(url.clone(), factory.clone()); + object_store_factories().insert(url.clone(), factory.clone()); + logstore_factories().insert(url.clone(), factory.clone()); } diff --git a/crates/gcp/src/storage.rs b/crates/gcp/src/storage.rs index 2dc160fe37..5572a716ed 100644 --- a/crates/gcp/src/storage.rs +++ b/crates/gcp/src/storage.rs @@ -1,13 +1,13 @@ //! GCP GCS storage backend. use bytes::Bytes; -use deltalake_core::storage::ObjectStoreRef; +use deltalake_core::logstore::ObjectStoreRef; use deltalake_core::Path; use futures::stream::BoxStream; use object_store::{MultipartUpload, PutMultipartOpts, PutPayload}; use std::ops::Range; -use deltalake_core::storage::object_store::{ +use deltalake_core::logstore::object_store::{ GetOptions, GetResult, ListResult, ObjectMeta, ObjectStore, PutOptions, PutResult, Result as ObjectStoreResult, }; @@ -57,7 +57,7 @@ impl ObjectStore for GcsStorageBackend { self.inner.get_opts(location, options).await } - async fn get_range(&self, location: &Path, range: Range) -> ObjectStoreResult { + async fn get_range(&self, location: &Path, range: Range) -> ObjectStoreResult { self.inner.get_range(location, range).await } @@ -69,7 +69,7 @@ impl ObjectStore for GcsStorageBackend { self.inner.delete(location).await } - fn list(&self, prefix: Option<&Path>) -> BoxStream<'_, ObjectStoreResult> { + fn list(&self, prefix: Option<&Path>) -> BoxStream<'static, ObjectStoreResult> { self.inner.list(prefix) } @@ -77,7 +77,7 @@ impl ObjectStore for GcsStorageBackend { &self, prefix: Option<&Path>, offset: &Path, - ) -> BoxStream<'_, ObjectStoreResult> { + ) -> BoxStream<'static, ObjectStoreResult> { self.inner.list_with_offset(prefix, offset) } diff --git a/crates/gcp/tests/context.rs b/crates/gcp/tests/context.rs index c095a27e3f..7174323408 100644 --- a/crates/gcp/tests/context.rs +++ b/crates/gcp/tests/context.rs @@ -129,6 +129,7 @@ pub mod gs_cli { .wait() } + #[allow(unused)] pub fn delete_bucket(container_name: impl AsRef) -> std::io::Result { let endpoint = std::env::var("GOOGLE_ENDPOINT_URL") .expect("variable GOOGLE_ENDPOINT_URL must be set to connect to GCS Emulator"); diff --git a/crates/hdfs/Cargo.toml b/crates/hdfs/Cargo.toml index 4dcc688f6d..90f60becad 100644 --- a/crates/hdfs/Cargo.toml +++ b/crates/hdfs/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "deltalake-hdfs" -version = "0.9.0" +version = "0.10.1" authors.workspace = true keywords.workspace = true readme.workspace = true @@ -12,8 +12,8 @@ repository.workspace = true rust-version.workspace = true [dependencies] -deltalake-core = { version = "0.25.0", path = "../core" } -hdfs-native-object-store = "0.12" +deltalake-core = { version = "0.26.0", path = "../core"} +hdfs-native-object-store = "0.14" # workspace dependecies object_store = { workspace = true } diff --git a/crates/hdfs/src/lib.rs b/crates/hdfs/src/lib.rs index 808ab680a1..ae5a31bded 100644 --- a/crates/hdfs/src/lib.rs +++ b/crates/hdfs/src/lib.rs @@ -1,11 +1,15 @@ +use std::collections::HashMap; use std::sync::Arc; -use deltalake_core::logstore::{default_logstore, logstores, LogStore, LogStoreFactory}; -use deltalake_core::storage::{ - factories, url_prefix_handler, ObjectStoreFactory, ObjectStoreRef, StorageOptions, +use deltalake_core::logstore::{ + default_logstore, logstore_factories, DeltaIOStorageBackend, LogStore, LogStoreFactory, + StorageConfig, }; +use deltalake_core::logstore::{object_store_factories, ObjectStoreFactory, ObjectStoreRef}; use deltalake_core::{DeltaResult, Path}; use hdfs_native_object_store::HdfsObjectStore; +use object_store::RetryConfig; +use tokio::runtime::Handle; use url::Url; #[derive(Clone, Default, Debug)] @@ -15,25 +19,36 @@ impl ObjectStoreFactory for HdfsFactory { fn parse_url_opts( &self, url: &Url, - options: &StorageOptions, + options: &HashMap, + _retry: &RetryConfig, + handle: Option, ) -> DeltaResult<(ObjectStoreRef, Path)> { - let store: ObjectStoreRef = Arc::new(HdfsObjectStore::with_config( - url.as_str(), - options.0.clone(), - )?); + let mut store: ObjectStoreRef = + Arc::new(HdfsObjectStore::with_config(url.as_str(), options.clone())?); + + // HDFS doesn't have the spawnService, so we still wrap it in the old io storage backend (not as optimal though) + if let Some(handle) = handle { + store = Arc::new(DeltaIOStorageBackend::new(store, handle)); + }; let prefix = Path::parse(url.path())?; - Ok((url_prefix_handler(store, prefix.clone()), prefix)) + Ok((store, prefix)) } } impl LogStoreFactory for HdfsFactory { fn with_options( &self, - store: ObjectStoreRef, + prefixed_store: ObjectStoreRef, + root_store: ObjectStoreRef, location: &Url, - options: &StorageOptions, + options: &StorageConfig, ) -> DeltaResult> { - Ok(default_logstore(store, location, options)) + Ok(default_logstore( + prefixed_store, + root_store, + location, + options, + )) } } @@ -42,7 +57,24 @@ pub fn register_handlers(_additional_prefixes: Option) { let factory = Arc::new(HdfsFactory {}); for scheme in ["hdfs", "viewfs"].iter() { let url = Url::parse(&format!("{scheme}://")).unwrap(); - factories().insert(url.clone(), factory.clone()); - logstores().insert(url.clone(), factory.clone()); + object_store_factories().insert(url.clone(), factory.clone()); + logstore_factories().insert(url.clone(), factory.clone()); + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_parse_url_opts() -> DeltaResult<()> { + let factory = HdfsFactory::default(); + let _ = factory.parse_url_opts( + &Url::parse("hdfs://localhost:9000").expect("Failed to parse hdfs://"), + &HashMap::default(), + &RetryConfig::default(), + None, + )?; + Ok(()) } } diff --git a/crates/lakefs/Cargo.toml b/crates/lakefs/Cargo.toml index 87a8954a4d..082cab71b6 100644 --- a/crates/lakefs/Cargo.toml +++ b/crates/lakefs/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "deltalake-lakefs" -version = "0.8.0" +version = "0.9.2" authors.workspace = true keywords.workspace = true readme.workspace = true @@ -12,7 +12,7 @@ repository.workspace = true rust-version.workspace = true [dependencies] -deltalake-core = { version = "0.25.0", path = "../core", features = ["cloud"] } +deltalake-core = { version = "0.26.0", path = "../core" } # workspace dependencies async-trait = { workspace = true } bytes = { workspace = true } @@ -29,11 +29,11 @@ dashmap = "6" serde = { workspace = true, features = ["derive"] } serde_json = { workspace = true } reqwest = {version = "0.12", default-features = false, features = ["http2", "json", "rustls-tls-native-roots"]} -http = "1.0.0" +http = "1.0" delta_kernel = { workspace = true, features = [] } [dev-dependencies] -deltalake-core = { path = "../core", features = ["datafusion"] } +deltalake-core = { path = "../core" } chrono = { workspace = true } serial_test = "3" deltalake-test = { path = "../test" } diff --git a/crates/lakefs/src/client.rs b/crates/lakefs/src/client.rs index 9647d086c8..37ae404f33 100644 --- a/crates/lakefs/src/client.rs +++ b/crates/lakefs/src/client.rs @@ -1,10 +1,10 @@ use dashmap::DashMap; -use deltalake_core::operations::transaction::TransactionError; +use deltalake_core::kernel::transaction::TransactionError; use deltalake_core::DeltaResult; use reqwest::Client; use reqwest::StatusCode; use serde::Deserialize; -use serde_json::json; +use serde_json::{json, Value}; use tracing::debug; use url::Url; use uuid::Uuid; @@ -195,7 +195,7 @@ impl LakeFSClient { "squash_merge": true, }); - debug!("Merging LakeFS, source `{transaction_branch}` into target `{transaction_branch}` in repo: {repo}"); + debug!("Merging LakeFS, source `{transaction_branch}` into target `{target_branch}` in repo: {repo}"); let response = self .http_client .post(&request_url) @@ -223,6 +223,56 @@ impl LakeFSClient { } } + pub async fn has_changes( + &self, + repo: &str, + base_branch: &str, + compare_branch: &str, + ) -> Result { + let request_url = format!( + "{}/api/v1/repositories/{repo}/refs/{base_branch}/diff/{compare_branch}", + self.config.host + ); + + debug!("Checking for changes from `{base_branch}` to `{compare_branch}` in repo: {repo}"); + let response = self + .http_client + .get(&request_url) + .basic_auth(&self.config.username, Some(&self.config.password)) + .send() + .await + .map_err(|e| LakeFSOperationError::HttpRequestFailed { source: e })?; + + match response.status() { + StatusCode::OK => { + // Parse the response to check if there are any differences + #[derive(Deserialize, Debug)] + struct DiffResponse { + results: Vec, + } + + let diff: DiffResponse = response + .json() + .await + .map_err(|e| LakeFSOperationError::HttpRequestFailed { source: e })?; + + // If there are any results in the diff, there are changes + Ok(!diff.results.is_empty()) + } + StatusCode::UNAUTHORIZED => Err(LakeFSOperationError::UnauthorizedAction.into()), + _ => { + let error: LakeFSErrorResponse = + response + .json() + .await + .unwrap_or_else(|_| LakeFSErrorResponse { + message: "Unknown error occurred.".to_string(), + }); + Err(LakeFSOperationError::MergeFailed(error.message).into()) + } + } + } + pub fn set_transaction(&self, id: Uuid, branch: String) { self.transactions.insert(id, branch); debug!("{}", format!("LakeFS Transaction `{id}` has been set.")); @@ -435,4 +485,51 @@ mod tests { let result = client.get_transaction(transaction_id); assert!(result.is_err()); } + + #[test] + fn test_has_changes() { + // Test cases with different parameters + let test_cases = vec![ + ("with_changes", r#"{"results": [{"some": "change"}]}"#, true), + ("without_changes", r#"{"results": []}"#, false), + ]; + + for (test_name, response_body, expected_has_changes) in test_cases { + let mut server = mockito::Server::new(); + let mock = server + .mock( + "GET", + "/api/v1/repositories/test_repo/refs/base_branch/diff/compare_branch", + ) + .with_status(StatusCode::OK.as_u16().into()) + .with_body(response_body) + .create(); + + let config = LakeFSConfig::new( + server.url(), + "test_user".to_string(), + "test_pass".to_string(), + ); + let client = LakeFSClient::with_config(config); + + let result = rt().block_on(async { + client + .has_changes("test_repo", "base_branch", "compare_branch") + .await + }); + + assert!( + result.is_ok(), + "Test case '{}' failed: API call returned error", + test_name + ); + let has_changes = result.unwrap(); + assert_eq!( + has_changes, expected_has_changes, + "Test case '{}' failed: expected has_changes to be {}", + test_name, expected_has_changes + ); + mock.assert(); + } + } } diff --git a/crates/lakefs/src/errors.rs b/crates/lakefs/src/errors.rs index f48c9d173a..ff994f8c1a 100644 --- a/crates/lakefs/src/errors.rs +++ b/crates/lakefs/src/errors.rs @@ -1,6 +1,6 @@ //! Errors for LakeFS log store -use deltalake_core::operations::transaction::TransactionError; +use deltalake_core::kernel::transaction::TransactionError; use deltalake_core::DeltaTableError; use reqwest::Error; diff --git a/crates/lakefs/src/execute.rs b/crates/lakefs/src/execute.rs index 73ea55a976..24fae5cab5 100644 --- a/crates/lakefs/src/execute.rs +++ b/crates/lakefs/src/execute.rs @@ -1,6 +1,8 @@ use async_trait::async_trait; use deltalake_core::{ - logstore::LogStoreRef, operations::CustomExecuteHandler, DeltaResult, DeltaTableError, + logstore::{LogStore as _, LogStoreRef}, + operations::CustomExecuteHandler, + DeltaResult, DeltaTableError, }; use tracing::debug; use uuid::Uuid; @@ -28,7 +30,7 @@ impl CustomExecuteHandler for LakeFSCustomExecuteHandler { if let Some(lakefs_store) = log_store.clone().as_any().downcast_ref::() { let (repo, _, _) = lakefs_store .client - .decompose_url(lakefs_store.config.location.to_string()); + .decompose_url(lakefs_store.config().location.to_string()); let result = lakefs_store .client .delete_branch(repo, lakefs_store.client.get_transaction(operation_id)?) @@ -91,10 +93,10 @@ mod tests { use crate::register_handlers; use super::*; - use deltalake_core::{logstore::logstore_for, storage::ObjectStoreRegistry}; + use deltalake_core::logstore::{logstore_for, ObjectStoreRegistry, StorageConfig}; use http::StatusCode; use maplit::hashmap; - use std::{collections::HashMap, sync::OnceLock}; + use std::sync::OnceLock; use tokio::runtime::Runtime; use url::Url; use uuid::Uuid; @@ -108,7 +110,9 @@ mod tests { "SECRET_ACCESS_KEY".to_string() => "options_key".to_string(), "REGION".to_string() => "options_key".to_string() }; - logstore_for(location, raw_options, None).unwrap() + + let storage_config = StorageConfig::parse_options(raw_options).unwrap(); + logstore_for(location, storage_config).unwrap() } #[inline] @@ -142,7 +146,7 @@ mod tests { .downcast_ref::() { assert!(lakefs_store - .storage + .prefixed_registry .get_store( &Url::parse(format!("lakefs://repo/delta-tx-{operation_id}/table").as_str()) .unwrap() @@ -183,7 +187,7 @@ mod tests { .downcast_ref::() { assert!(lakefs_store - .storage + .prefixed_registry .get_store( &Url::parse(format!("lakefs://repo/delta-tx-{operation_id}/table").as_str()) .unwrap() @@ -259,6 +263,16 @@ mod tests { .with_status(StatusCode::CREATED.as_u16().into()) .create(); + let diff_mock = server + .mock( + "GET", + format!("/api/v1/repositories/repo/refs/branch/diff/delta-tx-{operation_id}") + .as_str(), + ) + .with_status(StatusCode::OK.as_u16().into()) + .with_body(r#"{"results": [{"some": "change"}]}"#) + .create(); + let merge_branch_mock = server .mock( "POST", @@ -293,6 +307,7 @@ mod tests { }); create_commit_mock.assert(); + diff_mock.assert(); merge_branch_mock.assert(); delete_branch_mock.assert(); assert!(result.is_ok()); @@ -310,8 +325,8 @@ mod tests { #[tokio::test] async fn test_execute_error_with_invalid_log_store() { - let location = Url::parse("memory://table").unwrap(); - let invalid_default_store = logstore_for(location, HashMap::default(), None).unwrap(); + let location = Url::parse("memory:///table").unwrap(); + let invalid_default_store = logstore_for(location, StorageConfig::default()).unwrap(); let handler = LakeFSCustomExecuteHandler {}; let operation_id = Uuid::new_v4(); @@ -365,8 +380,8 @@ mod tests { async fn test_noop_commit_hook_executor() { // When file operations is false, the commit hook executor is a noop, since we don't need // to create any branches, or commit and merge them back. - let location = Url::parse("memory://table").unwrap(); - let invalid_default_store = logstore_for(location, HashMap::default(), None).unwrap(); + let location = Url::parse("memory:///table").unwrap(); + let invalid_default_store = logstore_for(location, StorageConfig::default()).unwrap(); let handler = LakeFSCustomExecuteHandler {}; let operation_id = Uuid::new_v4(); diff --git a/crates/lakefs/src/lib.rs b/crates/lakefs/src/lib.rs index e0417e2fe3..0b75cb986d 100644 --- a/crates/lakefs/src/lib.rs +++ b/crates/lakefs/src/lib.rs @@ -8,9 +8,9 @@ pub mod errors; pub mod execute; pub mod logstore; pub mod storage; -use deltalake_core::logstore::{logstores, LogStore, LogStoreFactory}; -use deltalake_core::storage::{factories, url_prefix_handler, ObjectStoreRef, StorageOptions}; -use deltalake_core::{DeltaResult, Path}; +use deltalake_core::logstore::{logstore_factories, LogStore, LogStoreFactory}; +use deltalake_core::logstore::{object_store_factories, ObjectStoreRef, StorageConfig}; +use deltalake_core::DeltaResult; pub use execute::LakeFSCustomExecuteHandler; use logstore::lakefs_logstore; use std::sync::Arc; @@ -27,14 +27,14 @@ impl S3StorageOptionsConversion for LakeFSLogStoreFactory {} impl LogStoreFactory for LakeFSLogStoreFactory { fn with_options( &self, - store: ObjectStoreRef, + prefixed_store: ObjectStoreRef, + root_store: ObjectStoreRef, location: &Url, - options: &StorageOptions, + config: &StorageConfig, ) -> DeltaResult> { - let options = self.with_env_s3(options); - let store = url_prefix_handler(store, Path::parse(location.path())?); + let options = StorageConfig::parse_options(self.with_env_s3(&config.raw.clone()))?; debug!("LakeFSLogStoreFactory has been asked to create a LogStore"); - lakefs_logstore(store, location, &options) + lakefs_logstore(prefixed_store, root_store, location, &options) } } @@ -44,6 +44,6 @@ pub fn register_handlers(_additional_prefixes: Option) { let log_stores = Arc::new(LakeFSLogStoreFactory::default()); let scheme = "lakefs"; let url = Url::parse(&format!("{scheme}://")).unwrap(); - factories().insert(url.clone(), object_stores.clone()); - logstores().insert(url.clone(), log_stores.clone()); + object_store_factories().insert(url.clone(), object_stores.clone()); + logstore_factories().insert(url.clone(), log_stores.clone()); } diff --git a/crates/lakefs/src/logstore.rs b/crates/lakefs/src/logstore.rs index 11be99387a..aeb9496f8e 100644 --- a/crates/lakefs/src/logstore.rs +++ b/crates/lakefs/src/logstore.rs @@ -1,45 +1,42 @@ //! Default implementation of [`LakeFSLogStore`] for LakeFS - use std::sync::{Arc, OnceLock}; -use crate::client::LakeFSConfig; -use crate::errors::LakeFSConfigError; - -use super::client::LakeFSClient; use bytes::Bytes; -use deltalake_core::storage::{ +use deltalake_core::logstore::{ commit_uri_from_version, DefaultObjectStoreRegistry, ObjectStoreRegistry, }; -use deltalake_core::storage::{url_prefix_handler, DeltaIOStorageBackend, IORuntime}; -use deltalake_core::{logstore::*, DeltaTableError, Path}; use deltalake_core::{ - operations::transaction::TransactionError, - storage::{ObjectStoreRef, StorageOptions}, - DeltaResult, + kernel::transaction::TransactionError, logstore::ObjectStoreRef, DeltaResult, }; -use object_store::{Attributes, Error as ObjectStoreError, ObjectStore, PutOptions, TagSet}; +use deltalake_core::{logstore::*, DeltaTableError}; +use object_store::{Error as ObjectStoreError, ObjectStore, PutOptions}; use tracing::debug; use url::Url; use uuid::Uuid; +use super::client::LakeFSClient; +use crate::client::LakeFSConfig; +use crate::errors::LakeFSConfigError; + /// Return the [LakeFSLogStore] implementation with the provided configuration options pub fn lakefs_logstore( store: ObjectStoreRef, + root_store: ObjectStoreRef, location: &Url, - options: &StorageOptions, + options: &StorageConfig, ) -> DeltaResult> { let host = options - .0 + .raw .get("aws_endpoint") .ok_or(LakeFSConfigError::EndpointMissing)? .to_string(); let username = options - .0 + .raw .get("aws_access_key_id") .ok_or(LakeFSConfigError::UsernameCredentialMissing)? .to_string(); let password = options - .0 + .raw .get("aws_secret_access_key") .ok_or(LakeFSConfigError::PasswordCredentialMissing)? .to_string(); @@ -47,6 +44,7 @@ pub fn lakefs_logstore( let client = LakeFSClient::with_config(LakeFSConfig::new(host, username, password)); Ok(Arc::new(LakeFSLogStore::new( store, + root_store, LogStoreConfig { location: location.clone(), options: options.clone(), @@ -58,8 +56,9 @@ pub fn lakefs_logstore( /// Default [`LogStore`] implementation #[derive(Debug, Clone)] pub(crate) struct LakeFSLogStore { - pub(crate) storage: DefaultObjectStoreRegistry, - pub(crate) config: LogStoreConfig, + pub(crate) prefixed_registry: DefaultObjectStoreRegistry, + root_registry: DefaultObjectStoreRegistry, + config: LogStoreConfig, pub(crate) client: LakeFSClient, } @@ -68,13 +67,24 @@ impl LakeFSLogStore { /// /// # Arguments /// - /// * `storage` - A shared reference to an [`object_store::ObjectStore`] with "/" pointing at delta table root (i.e. where `_delta_log` is located). + /// * `prefixed_store` - A shared reference to an [`object_store::ObjectStore`] + /// with "/" pointing at delta table root (i.e. where `_delta_log` is located). + /// * `root_store` - A shared reference to an [`object_store::ObjectStore`] with "/" + /// pointing at root of the storage system. /// * `location` - A url corresponding to the storage location of `storage`. - pub fn new(storage: ObjectStoreRef, config: LogStoreConfig, client: LakeFSClient) -> Self { - let registry = DefaultObjectStoreRegistry::new(); - registry.register_store(&config.location, storage); + pub fn new( + prefixed_store: ObjectStoreRef, + root_store: ObjectStoreRef, + config: LogStoreConfig, + client: LakeFSClient, + ) -> Self { + let prefixed_registry = DefaultObjectStoreRegistry::new(); + prefixed_registry.register_store(&config.location, prefixed_store); + let root_registry = DefaultObjectStoreRegistry::new(); + root_registry.register_store(&config.location, root_store); Self { - storage: registry, + prefixed_registry, + root_registry, config, client, } @@ -82,37 +92,56 @@ impl LakeFSLogStore { /// Build a new object store for an URL using the existing storage options. After /// branch creation a new object store needs to be created for the branch uri - fn build_new_store(&self, url: &Url) -> DeltaResult { + fn build_new_store( + &self, + url: &Url, + io_runtime: Option, + ) -> DeltaResult { // turn location into scheme let scheme = Url::parse(&format!("{}://", url.scheme())) .map_err(|_| DeltaTableError::InvalidTableLocation(url.clone().into()))?; - if let Some(entry) = deltalake_core::storage::factories().get(&scheme) { + if let Some(entry) = self.config().object_store_factory().get(&scheme) { debug!("Creating new storage with storage provider for {scheme} ({url})"); - - let (store, _prefix) = entry - .value() - .parse_url_opts(url, &self.config().options.clone())?; + let (store, _prefix) = entry.value().parse_url_opts( + url, + &self.config().options.raw, + &self.config().options.retry, + io_runtime.map(|rt| rt.get_handle()), + )?; return Ok(store); } Err(DeltaTableError::InvalidTableLocation(url.to_string())) } fn register_object_store(&self, url: &Url, store: ObjectStoreRef) { - self.storage.register_store(url, store); + self.prefixed_registry.register_store(url, store); } - fn get_transaction_objectstore( - &self, - operation_id: Uuid, - ) -> DeltaResult<(String, ObjectStoreRef)> { - let (repo, _, table) = self.client.decompose_url(self.config.location.to_string()); + fn register_root_object_store(&self, url: &Url, store: ObjectStoreRef) { + self.root_registry.register_store(url, store); + } + + fn get_transaction_url(&self, operation_id: Uuid, base: String) -> DeltaResult { + let (repo, _, table) = self.client.decompose_url(base); let string_url = format!( "lakefs://{repo}/{}/{table}", self.client.get_transaction(operation_id)?, ); - let transaction_url = Url::parse(&string_url).unwrap(); - Ok((string_url, self.storage.get_store(&transaction_url)?)) + Ok(Url::parse(&string_url).unwrap()) + } + + fn get_transaction_objectstore( + &self, + operation_id: Uuid, + ) -> DeltaResult<(String, ObjectStoreRef, ObjectStoreRef)> { + let transaction_url = + self.get_transaction_url(operation_id, self.config.location.to_string())?; + Ok(( + transaction_url.clone().to_string(), + self.prefixed_registry.get_store(&transaction_url)?, + self.root_registry.get_store(&transaction_url)?, + )) } pub async fn pre_execute(&self, operation_id: Uuid) -> DeltaResult<()> { @@ -123,15 +152,15 @@ impl LakeFSLogStore { .await?; // Build new object store store using the new lakefs url - let txn_store = url_prefix_handler( - Arc::new(DeltaIOStorageBackend::new( - self.build_new_store(&lakefs_url)?, - IORuntime::default().get_handle(), - )) as ObjectStoreRef, - Path::parse(lakefs_url.path())?, + let txn_root_store = + self.build_new_store(&lakefs_url, self.config().options.runtime.clone())?; + let txn_store = Arc::new( + self.config + .decorate_store(txn_root_store.clone(), Some(&lakefs_url))?, ); // Register transaction branch as ObjectStore in log_store storages + self.register_root_object_store(&lakefs_url, txn_root_store); self.register_object_store(&lakefs_url, txn_store); // set transaction in client for easy retrieval @@ -140,50 +169,68 @@ impl LakeFSLogStore { } pub async fn commit_merge(&self, operation_id: Uuid) -> DeltaResult<()> { - let (transaction_url, _) = self.get_transaction_objectstore(operation_id)?; + let (transaction_url, _, _) = self.get_transaction_objectstore(operation_id)?; // Do LakeFS Commit let (repo, transaction_branch, table) = self.client.decompose_url(transaction_url); self.client .commit( - repo, - transaction_branch, + repo.clone(), + transaction_branch.clone(), format!("Delta file operations {{ table: {table}}}"), true, // Needs to be true, it could be a file operation but no logs were deleted. ) .await?; - // Try LakeFS Branch merge of transaction branch in source branch + // Get target branch information let (repo, target_branch, table) = self.client.decompose_url(self.config.location.to_string()); - match self + + // Check if there are any changes before attempting to merge + let has_changes = self .client - .merge( - repo, - target_branch, - self.client.get_transaction(operation_id)?, - 0, - format!("Finished delta file operations {{ table: {table}}}"), - true, // Needs to be true, it could be a file operation but no logs were deleted. - ) + .has_changes(&repo, &target_branch, &transaction_branch) .await - { - Ok(_) => { - let (repo, _, _) = self.client.decompose_url(self.config.location.to_string()); - self.client - .delete_branch(repo, self.client.get_transaction(operation_id)?) - .await?; - Ok(()) - } - // TODO: propagate better LakeFS errors. - Err(TransactionError::VersionAlreadyExists(_)) => { - Err(TransactionError::LogStoreError { - msg: "Merge Failed".to_string(), - source: Box::new(DeltaTableError::generic("Merge Failed")), - }) - } - Err(err) => Err(err), - }?; + .map_err(|e| DeltaTableError::generic(format!("Failed to check for changes: {}", e)))?; + + // Only perform merge if there are changes + if has_changes { + debug!("Changes detected, proceeding with merge"); + match self + .client + .merge( + repo, + target_branch, + self.client.get_transaction(operation_id)?, + 0, + format!("Finished delta file operations {{ table: {table}}}"), + true, // Needs to be true, it could be a file operation but no logs were deleted. + ) + .await + { + Ok(_) => { + // Merge successful + } + // TODO: propagate better LakeFS errors. + Err(TransactionError::VersionAlreadyExists(_)) => { + return Err(DeltaTableError::Transaction { + source: TransactionError::LogStoreError { + msg: "Merge Failed".to_string(), + source: Box::new(DeltaTableError::generic("Merge Failed")), + }, + }); + } + Err(err) => return Err(DeltaTableError::Transaction { source: err }), + }; + } else { + debug!("No changes detected, skipping merge"); + } + + // Always delete the transaction branch when done + let (repo, _, _) = self.client.decompose_url(self.config.location.to_string()); + self.client + .delete_branch(repo, self.client.get_transaction(operation_id)?) + .await?; self.client.clear_transaction(operation_id); Ok(()) @@ -197,7 +244,11 @@ impl LogStore for LakeFSLogStore { } async fn read_commit_entry(&self, version: i64) -> DeltaResult> { - read_commit_entry(&self.storage.get_store(&self.config.location)?, version).await + read_commit_entry( + &self.prefixed_registry.get_store(&self.config.location)?, + version, + ) + .await } /// Tries to commit a prepared commit file. Returns [`TransactionError`] @@ -211,12 +262,12 @@ impl LogStore for LakeFSLogStore { commit_or_bytes: CommitOrBytes, operation_id: Uuid, ) -> Result<(), TransactionError> { - let (transaction_url, store) = - self.get_transaction_objectstore(operation_id) - .map_err(|e| TransactionError::LogStoreError { - msg: e.to_string(), - source: Box::new(e), - })?; + let (transaction_url, store, _root_store) = self + .get_transaction_objectstore(operation_id) + .map_err(|e| TransactionError::LogStoreError { + msg: e.to_string(), + source: Box::new(e), + })?; match commit_or_bytes { CommitOrBytes::LogBytes(log_bytes) => { @@ -306,30 +357,44 @@ impl LogStore for LakeFSLogStore { get_latest_version(self, current_version).await } - async fn get_earliest_version(&self, current_version: i64) -> DeltaResult { - get_earliest_version(self, current_version).await - } - fn object_store(&self, operation_id: Option) -> Arc { match operation_id { Some(id) => { - let (_, store) = self.get_transaction_objectstore(id).unwrap_or_else(|_| panic!("The object_store registry inside LakeFSLogstore didn't have a store for operation_id {id} Something went wrong.")); + let (_, store, _) = self.get_transaction_objectstore(id).unwrap_or_else(|_| panic!("The object_store registry inside LakeFSLogstore didn't have a store for operation_id {id} Something went wrong.")); store } - _ => self.storage.get_store(&self.config.location).unwrap(), + _ => self + .prefixed_registry + .get_store(&self.config.location) + .unwrap(), + } + } + + fn root_object_store(&self, operation_id: Option) -> Arc { + match operation_id { + Some(id) => { + let (_, _, root_store) = self.get_transaction_objectstore(id).unwrap_or_else(|_| panic!("The object_store registry inside LakeFSLogstore didn't have a store for operation_id {id} Something went wrong.")); + root_store + } + _ => self.root_registry.get_store(&self.config.location).unwrap(), } } fn config(&self) -> &LogStoreConfig { &self.config } + + fn transaction_url(&self, operation_id: Uuid, base: &Url) -> DeltaResult { + self.get_transaction_url(operation_id, base.to_string()) + } } fn put_options() -> &'static PutOptions { static PUT_OPTS: OnceLock = OnceLock::new(); PUT_OPTS.get_or_init(|| PutOptions { mode: object_store::PutMode::Create, // Creates if file doesn't exists yet - tags: TagSet::default(), - attributes: Attributes::default(), + tags: Default::default(), + attributes: Default::default(), + extensions: Default::default(), }) } diff --git a/crates/lakefs/src/storage.rs b/crates/lakefs/src/storage.rs index 5774336827..52e917671d 100644 --- a/crates/lakefs/src/storage.rs +++ b/crates/lakefs/src/storage.rs @@ -1,15 +1,16 @@ //! LakeFS storage backend (internally S3). -use deltalake_core::storage::object_store::aws::AmazonS3ConfigKey; -use deltalake_core::storage::{ - limit_store_handler, ObjectStoreFactory, ObjectStoreRef, RetryConfigParse, StorageOptions, -}; +use deltalake_core::logstore::object_store::aws::AmazonS3ConfigKey; +use deltalake_core::logstore::{ObjectStoreFactory, ObjectStoreRef}; use deltalake_core::{DeltaResult, DeltaTableError, Path}; use object_store::aws::AmazonS3Builder; -use object_store::ObjectStoreScheme; +use object_store::client::SpawnedReqwestConnector; +use object_store::{ObjectStoreScheme, RetryConfig}; use std::collections::HashMap; use std::fmt::Debug; use std::str::FromStr; +use std::sync::Arc; +use tokio::runtime::Handle; use tracing::log::*; use url::Url; @@ -17,36 +18,31 @@ use url::Url; pub struct LakeFSObjectStoreFactory {} pub(crate) trait S3StorageOptionsConversion { - fn with_env_s3(&self, options: &StorageOptions) -> StorageOptions { - let mut options = StorageOptions( - options - .0 - .clone() - .into_iter() - .map(|(k, v)| { - if let Ok(config_key) = AmazonS3ConfigKey::from_str(&k.to_ascii_lowercase()) { - (config_key.as_ref().to_string(), v) - } else { - (k, v) - } - }) - .collect(), - ); + fn with_env_s3(&self, options: &HashMap) -> HashMap { + let mut options: HashMap = options + .clone() + .into_iter() + .map(|(k, v)| { + if let Ok(config_key) = AmazonS3ConfigKey::from_str(&k.to_ascii_lowercase()) { + (config_key.as_ref().to_string(), v) + } else { + (k, v) + } + }) + .collect(); for (os_key, os_value) in std::env::vars_os() { if let (Some(key), Some(value)) = (os_key.to_str(), os_value.to_str()) { if let Ok(config_key) = AmazonS3ConfigKey::from_str(&key.to_ascii_lowercase()) { - if !options.0.contains_key(config_key.as_ref()) { - options - .0 - .insert(config_key.as_ref().to_string(), value.to_string()); + if !options.contains_key(config_key.as_ref()) { + options.insert(config_key.as_ref().to_string(), value.to_string()); } } } } // Conditional put is supported in LakeFS since v1.47 - if !options.0.keys().any(|key| { + if !options.keys().any(|key| { let key = key.to_ascii_lowercase(); [ AmazonS3ConfigKey::ConditionalPut.as_ref(), @@ -54,7 +50,7 @@ pub(crate) trait S3StorageOptionsConversion { ] .contains(&key.as_str()) }) { - options.0.insert("conditional_put".into(), "etag".into()); + options.insert("conditional_put".into(), "etag".into()); } options } @@ -62,26 +58,23 @@ pub(crate) trait S3StorageOptionsConversion { impl S3StorageOptionsConversion for LakeFSObjectStoreFactory {} -impl RetryConfigParse for LakeFSObjectStoreFactory {} - impl ObjectStoreFactory for LakeFSObjectStoreFactory { fn parse_url_opts( &self, url: &Url, - storage_options: &StorageOptions, + storage_config: &HashMap, + retry: &RetryConfig, + handle: Option, ) -> DeltaResult<(ObjectStoreRef, Path)> { - let options = self.with_env_s3(storage_options); - // Convert LakeFS URI to equivalent S3 URI. let s3_url = url.to_string().replace("lakefs://", "s3://"); - let s3_url = Url::parse(&s3_url) .map_err(|_| DeltaTableError::InvalidTableLocation(url.clone().into()))?; // All S3-likes should start their builder the same way + let options = self.with_env_s3(storage_config); let config = options .clone() - .0 .into_iter() .filter_map(|(k, v)| { if let Ok(key) = AmazonS3ConfigKey::from_str(&k.to_ascii_lowercase()) { @@ -100,17 +93,18 @@ impl ObjectStoreFactory for LakeFSObjectStoreFactory { let mut builder = AmazonS3Builder::new().with_url(s3_url.to_string()); + if let Some(handle) = handle { + builder = builder.with_http_connector(SpawnedReqwestConnector::new(handle)); + } + for (key, value) in config.iter() { builder = builder.with_config(*key, value.clone()); } - let inner = builder - .with_retry(self.parse_retry_config(&options)?) - .build()?; + let store = builder.with_retry(retry.clone()).build()?; - let store = limit_store_handler(inner, &options); debug!("Initialized the object store: {store:?}"); - Ok((store, prefix)) + Ok((Arc::new(store), prefix)) } } @@ -160,13 +154,12 @@ mod tests { std::env::set_var("ENDPOINT", "env_key"); std::env::set_var("SECRET_ACCESS_KEY", "env_key"); std::env::set_var("REGION", "env_key"); - let combined_options = - LakeFSObjectStoreFactory {}.with_env_s3(&StorageOptions(raw_options)); + let combined_options = LakeFSObjectStoreFactory {}.with_env_s3(&raw_options); // Four and then the conditional_put built-in - assert_eq!(combined_options.0.len(), 5); + assert_eq!(combined_options.len(), 5); - for (key, v) in combined_options.0 { + for (key, v) in combined_options { if key != "conditional_put" { assert_eq!(v, "env_key"); } @@ -190,10 +183,9 @@ mod tests { std::env::set_var("aws_secret_access_key", "env_key"); std::env::set_var("aws_region", "env_key"); - let combined_options = - LakeFSObjectStoreFactory {}.with_env_s3(&StorageOptions(raw_options)); + let combined_options = LakeFSObjectStoreFactory {}.with_env_s3(&raw_options); - for (key, v) in combined_options.0 { + for (key, v) in combined_options { if key != "conditional_put" { assert_eq!(v, "options_key"); } diff --git a/crates/lakefs/tests/context.rs b/crates/lakefs/tests/context.rs index cb04db8bdf..0bf46e1d4b 100644 --- a/crates/lakefs/tests/context.rs +++ b/crates/lakefs/tests/context.rs @@ -44,8 +44,8 @@ impl StorageIntegration for LakeFSIntegration { fn copy_directory(&self, source: &str, destination: &str) -> std::io::Result { println!( - "Copy directory called with {source} {}", - format!("{}/{destination}", self.root_uri()) + "Copy directory called with {source} {}/{destination}", + self.root_uri() ); let lakectl = which("lakectl").expect("Failed to find lakectl executable"); diff --git a/crates/mount/Cargo.toml b/crates/mount/Cargo.toml index 1ea3605071..688f5523f5 100644 --- a/crates/mount/Cargo.toml +++ b/crates/mount/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "deltalake-mount" -version = "0.9.0" +version = "0.10.0" authors.workspace = true keywords.workspace = true readme.workspace = true @@ -12,9 +12,7 @@ repository.workspace = true rust-version.workspace = true [dependencies] -deltalake-core = { version = "0.25.0", path = "../core", features = [ - "datafusion", -] } +deltalake-core = { version = "0.26.0", path = "../core" } errno = "0.3" # workspace depenndecies diff --git a/crates/mount/src/file.rs b/crates/mount/src/file.rs index 9c87bf3b2f..8edcf3bdc9 100644 --- a/crates/mount/src/file.rs +++ b/crates/mount/src/file.rs @@ -12,7 +12,6 @@ use object_store::{ use object_store::{MultipartUpload, PutMode, PutMultipartOpts, PutPayload}; use std::ops::Range; use std::sync::Arc; -use url::Url; pub(crate) const STORE_NAME: &str = "MountObjectStore"; @@ -108,42 +107,20 @@ impl From for ObjectStoreError { #[derive(Debug)] pub struct MountFileStorageBackend { inner: Arc, - root_url: Arc, } impl MountFileStorageBackend { /// Creates a new MountFileStorageBackend. - pub fn try_new(path: impl AsRef) -> ObjectStoreResult { + pub fn try_new() -> ObjectStoreResult { Ok(Self { - root_url: Arc::new(Self::path_to_root_url(path.as_ref())?), - inner: Arc::new(LocalFileSystem::new_with_prefix(path)?), - }) - } - - fn path_to_root_url(path: &std::path::Path) -> ObjectStoreResult { - let root_path = - std::fs::canonicalize(path).map_err(|e| object_store::Error::InvalidPath { - source: object_store::path::Error::Canonicalize { - path: path.into(), - source: e, - }, - })?; - - Url::from_file_path(root_path).map_err(|_| object_store::Error::InvalidPath { - source: object_store::path::Error::InvalidPath { path: path.into() }, + inner: Arc::new(LocalFileSystem::new()), }) } /// Return an absolute filesystem path of the given location fn path_to_filesystem(&self, location: &ObjectStorePath) -> String { - let mut url = self.root_url.as_ref().clone(); - url.path_segments_mut() - .expect("url path") - // technically not necessary as Path ignores empty segments - // but avoids creating paths with "//" which look odd in error messages. - .pop_if_empty() - .extend(location.parts()); - + let mut url = url::Url::parse("file:///").unwrap(); + url.set_path(location.as_ref()); url.to_file_path().unwrap().to_str().unwrap().to_owned() } } @@ -191,7 +168,7 @@ impl ObjectStore for MountFileStorageBackend { async fn get_range( &self, location: &ObjectStorePath, - range: Range, + range: Range, ) -> ObjectStoreResult { self.inner.get_range(location, range).await } @@ -207,7 +184,7 @@ impl ObjectStore for MountFileStorageBackend { fn list( &self, prefix: Option<&ObjectStorePath>, - ) -> BoxStream<'_, ObjectStoreResult> { + ) -> BoxStream<'static, ObjectStoreResult> { self.inner.list(prefix) } @@ -215,7 +192,7 @@ impl ObjectStore for MountFileStorageBackend { &self, prefix: Option<&ObjectStorePath>, offset: &ObjectStorePath, - ) -> BoxStream<'_, ObjectStoreResult> { + ) -> BoxStream<'static, ObjectStoreResult> { self.inner.list_with_offset(prefix, offset) } diff --git a/crates/mount/src/lib.rs b/crates/mount/src/lib.rs index 31b8b0637e..d36937bd6a 100644 --- a/crates/mount/src/lib.rs +++ b/crates/mount/src/lib.rs @@ -2,12 +2,16 @@ use std::collections::HashMap; use std::str::FromStr; use std::sync::Arc; -use deltalake_core::logstore::{default_logstore, logstores, LogStore, LogStoreFactory}; -use deltalake_core::storage::{ - factories, str_is_truthy, ObjectStoreFactory, ObjectStoreRef, StorageOptions, +use deltalake_core::logstore::DeltaIOStorageBackend; +use deltalake_core::logstore::{ + config::str_is_truthy, default_logstore, logstore_factories, object_store_factories, LogStore, + LogStoreFactory, ObjectStoreFactory, ObjectStoreRef, StorageConfig, }; use deltalake_core::{DeltaResult, DeltaTableError, Path}; use object_store::local::LocalFileSystem; +use object_store::DynObjectStore; +use object_store::RetryConfig; +use tokio::runtime::Handle; use url::Url; mod config; @@ -18,10 +22,9 @@ trait MountOptions { fn as_mount_options(&self) -> HashMap; } -impl MountOptions for StorageOptions { +impl MountOptions for HashMap { fn as_mount_options(&self) -> HashMap { - self.0 - .iter() + self.iter() .filter_map(|(key, value)| { Some(( config::MountConfigKey::from_str(&key.to_ascii_lowercase()).ok()?, @@ -39,7 +42,9 @@ impl ObjectStoreFactory for MountFactory { fn parse_url_opts( &self, url: &Url, - options: &StorageOptions, + options: &HashMap, + _retry: &RetryConfig, + handle: Option, ) -> DeltaResult<(ObjectStoreRef, Path)> { let config = config::MountConfigHelper::try_new(options.as_mount_options())?.build()?; @@ -49,45 +54,53 @@ impl ObjectStoreFactory for MountFactory { .unwrap_or(&String::new()), ); - match url.scheme() { + let (mut store, prefix) = match url.scheme() { "dbfs" => { if !allow_unsafe_rename { // Just let the user know that they need to set the allow_unsafe_rename option return Err(error::Error::AllowUnsafeRenameNotSpecified.into()); } // We need to convert the dbfs url to a file url - let new_url = Url::parse(&format!("file:///dbfs{}", url.path())).unwrap(); - let store = Arc::new(file::MountFileStorageBackend::try_new( - new_url.to_file_path().unwrap(), - )?) as ObjectStoreRef; + Url::parse(&format!("file:///dbfs{}", url.path())).unwrap(); + let store = Arc::new(file::MountFileStorageBackend::try_new()?) as ObjectStoreRef; Ok((store, Path::from("/"))) } "file" => { if allow_unsafe_rename { - let store = Arc::new(file::MountFileStorageBackend::try_new( - url.to_file_path().unwrap(), - )?) as ObjectStoreRef; - Ok((store, Path::from("/"))) + let store = + Arc::new(file::MountFileStorageBackend::try_new()?) as ObjectStoreRef; + let prefix = Path::from_filesystem_path(url.to_file_path().unwrap())?; + Ok((store, prefix)) } else { - let store = Arc::new(LocalFileSystem::new_with_prefix( - url.to_file_path().unwrap(), - )?) as ObjectStoreRef; - Ok((store, Path::from("/"))) + let store = Arc::new(LocalFileSystem::new()) as ObjectStoreRef; + let prefix = Path::from_filesystem_path(url.to_file_path().unwrap())?; + Ok((store, prefix)) } } _ => Err(DeltaTableError::InvalidTableLocation(url.clone().into())), + }?; + + if let Some(handle) = handle { + store = Arc::new(DeltaIOStorageBackend::new(store, handle)) as Arc; } + Ok((store, prefix)) } } impl LogStoreFactory for MountFactory { fn with_options( &self, - store: ObjectStoreRef, + prefixed_store: ObjectStoreRef, + root_store: ObjectStoreRef, location: &Url, - options: &StorageOptions, + options: &StorageConfig, ) -> DeltaResult> { - Ok(default_logstore(store, location, options)) + Ok(default_logstore( + prefixed_store, + root_store, + location, + options, + )) } } @@ -96,7 +109,7 @@ pub fn register_handlers(_additional_prefixes: Option) { let factory = Arc::new(MountFactory {}); for scheme in ["dbfs", "file"].iter() { let url = Url::parse(&format!("{scheme}://")).unwrap(); - factories().insert(url.clone(), factory.clone()); - logstores().insert(url.clone(), factory.clone()); + object_store_factories().insert(url.clone(), factory.clone()); + logstore_factories().insert(url.clone(), factory.clone()); } } diff --git a/crates/mount/tests/integration.rs b/crates/mount/tests/integration.rs index 14fcbcdc95..729d64261b 100644 --- a/crates/mount/tests/integration.rs +++ b/crates/mount/tests/integration.rs @@ -11,7 +11,7 @@ static TEST_PREFIXES: &[&str] = &["my table", "你好/😊"]; #[tokio::test] #[serial] -async fn test_integration_local() -> TestResult { +async fn test_integration_mount() -> TestResult { let context = IntegrationContext::new(Box::::default())?; test_read_tables(&context).await?; diff --git a/crates/sql/src/logical_plan.rs b/crates/sql/src/logical_plan.rs index c33b730252..349a89b271 100644 --- a/crates/sql/src/logical_plan.rs +++ b/crates/sql/src/logical_plan.rs @@ -90,11 +90,6 @@ impl UserDefinedLogicalNodeCore for DeltaStatement { write!(f, "{}", self.display()) } - fn from_template(&self, exprs: &[Expr], inputs: &[LogicalPlan]) -> Self { - self.with_exprs_and_inputs(exprs.to_vec(), inputs.to_vec()) - .unwrap() - } - fn with_exprs_and_inputs( &self, exprs: Vec, diff --git a/crates/sql/src/parser.rs b/crates/sql/src/parser.rs index 3ae9b51d51..98d1b2b68a 100644 --- a/crates/sql/src/parser.rs +++ b/crates/sql/src/parser.rs @@ -167,7 +167,7 @@ impl<'a> DeltaParser<'a> { Token::Word(w) => match w.keyword { Keyword::RETAIN => { self.parser.next_token(); - let retention_hours = match self.parser.parse_number_value()? { + let retention_hours = match self.parser.parse_number_value()?.value { Value::Number(value_str, _) => value_str .parse() .map_err(|_| ParserError::ParserError(format!("Unexpected token {w}"))), @@ -223,7 +223,7 @@ impl<'a> DeltaParser<'a> { #[cfg(test)] mod tests { use super::*; - use datafusion_sql::sqlparser::ast::Ident; + use datafusion_sql::sqlparser::ast::{Ident, ObjectNamePart}; use datafusion_sql::sqlparser::tokenizer::Span; fn expect_parse_ok(sql: &str, expected: Statement) -> Result<(), ParserError> { @@ -240,44 +240,44 @@ mod tests { #[test] fn test_parse_vacuum() { let stmt = Statement::Vacuum(VacuumStatement { - table: ObjectName(vec![Ident { + table: ObjectName(vec![ObjectNamePart::Identifier(Ident { value: "data_table".to_string(), quote_style: None, span: Span::empty(), - }]), + })]), retention_hours: None, dry_run: false, }); assert!(expect_parse_ok("VACUUM data_table", stmt).is_ok()); let stmt = Statement::Vacuum(VacuumStatement { - table: ObjectName(vec![Ident { + table: ObjectName(vec![ObjectNamePart::Identifier(Ident { value: "data_table".to_string(), quote_style: None, span: Span::empty(), - }]), + })]), retention_hours: Some(10), dry_run: false, }); assert!(expect_parse_ok("VACUUM data_table RETAIN 10 HOURS", stmt).is_ok()); let stmt = Statement::Vacuum(VacuumStatement { - table: ObjectName(vec![Ident { + table: ObjectName(vec![ObjectNamePart::Identifier(Ident { value: "data_table".to_string(), quote_style: None, span: Span::empty(), - }]), + })]), retention_hours: Some(10), dry_run: true, }); assert!(expect_parse_ok("VACUUM data_table RETAIN 10 HOURS DRY RUN", stmt).is_ok()); let stmt = Statement::Vacuum(VacuumStatement { - table: ObjectName(vec![Ident { + table: ObjectName(vec![ObjectNamePart::Identifier(Ident { value: "data_table".to_string(), quote_style: None, span: Span::empty(), - }]), + })]), retention_hours: None, dry_run: true, }); diff --git a/crates/sql/src/planner.rs b/crates/sql/src/planner.rs index a95a43bb05..d34437372b 100644 --- a/crates/sql/src/planner.rs +++ b/crates/sql/src/planner.rs @@ -182,7 +182,7 @@ mod tests { fn test_planner() { test_statement( "SELECT * FROM table1", - &["Projection: *", " TableScan: table1"], + &["Projection: table1.column1", " TableScan: table1"], ); test_statement("VACUUM table1", &["Vacuum: table1 dry_run=false"]); diff --git a/crates/test/Cargo.toml b/crates/test/Cargo.toml index 2d13097c3e..c986f25f0e 100644 --- a/crates/test/Cargo.toml +++ b/crates/test/Cargo.toml @@ -1,23 +1,36 @@ [package] name = "deltalake-test" -version = "0.8.0" +version = "0.9.0" edition = "2021" publish = false [dependencies] +deltalake-core = { version = "0.26.0", path = "../core", features = [ + "integration_test", +] } + +arrow-array = { workspace = true, features = ["chrono-tz"] } +arrow-cast = { workspace = true } +arrow-ord = { workspace = true } +arrow-schema = { workspace = true, features = ["serde"] } +arrow-select = { workspace = true } +parquet = { workspace = true, features = ["async", "object_store"] } + bytes = { workspace = true } chrono = { workspace = true, default-features = false, features = ["clock"] } -deltalake-core = { version = "0.25.0", path = "../core" } +delta_kernel = { workspace = true } +object_store = { workspace = true } +serde = { workspace = true, features = ["derive"] } +serde_json = { workspace = true } +thiserror = { workspace = true } +url = { workspace = true } + dotenvy = "0" fs_extra = "1.3.0" futures = { version = "0.3" } -object_store = { workspace = true } rand = "0.8" -serde = { workspace = true, features = ["derive"] } -serde_json = { workspace = true } tempfile = "3" tokio = { version = "1", features = ["macros", "rt-multi-thread"] } [features] default = [] -datafusion = ["deltalake-core/datafusion"] diff --git a/crates/test/src/acceptance/data.rs b/crates/test/src/acceptance/data.rs new file mode 100644 index 0000000000..0d04273cad --- /dev/null +++ b/crates/test/src/acceptance/data.rs @@ -0,0 +1,130 @@ +use std::{path::Path, sync::Arc}; + +use arrow_array::{Array, RecordBatch}; +use arrow_ord::sort::{lexsort_to_indices, SortColumn}; +use arrow_schema::{DataType, Schema}; +use arrow_select::{concat::concat_batches, take::take}; +use delta_kernel::DeltaResult; +use futures::{stream::TryStreamExt, StreamExt}; +use object_store::{local::LocalFileSystem, ObjectStore}; +use parquet::arrow::async_reader::{ParquetObjectReader, ParquetRecordBatchStreamBuilder}; + +use super::TestCaseInfo; +use crate::TestResult; + +pub async fn read_golden(path: &Path, _version: Option<&str>) -> DeltaResult { + let expected_root = path.join("expected").join("latest").join("table_content"); + let store = Arc::new(LocalFileSystem::new_with_prefix(&expected_root)?); + let files: Vec<_> = store.list(None).try_collect().await?; + let mut batches = vec![]; + let mut schema = None; + for meta in files.into_iter() { + if let Some(ext) = meta.location.extension() { + if ext == "parquet" { + let reader = ParquetObjectReader::new(store.clone(), meta.location); + let builder = ParquetRecordBatchStreamBuilder::new(reader).await?; + if schema.is_none() { + schema = Some(builder.schema().clone()); + } + let mut stream = builder.build()?; + while let Some(batch) = stream.next().await { + batches.push(batch?); + } + } + } + } + let all_data = concat_batches(&schema.unwrap(), &batches)?; + Ok(all_data) +} + +pub fn sort_record_batch(batch: RecordBatch) -> DeltaResult { + // Sort by all columns + let mut sort_columns = vec![]; + for col in batch.columns() { + match col.data_type() { + DataType::Struct(_) | DataType::List(_) | DataType::Map(_, _) => { + // can't sort structs, lists, or maps + } + _ => sort_columns.push(SortColumn { + values: col.clone(), + options: None, + }), + } + } + let indices = lexsort_to_indices(&sort_columns, None)?; + let columns = batch + .columns() + .iter() + .map(|c| take(c, &indices, None).unwrap()) + .collect(); + Ok(RecordBatch::try_new(batch.schema(), columns)?) +} + +// Ensure that two schema have the same field names, and dict_id/ordering. +// We ignore: +// - data type: This is checked already in `assert_columns_match` +// - nullability: parquet marks many things as nullable that we don't in our schema +// - metadata: because that diverges from the real data to the golden tabled data +fn assert_schema_fields_match(schema: &Schema, golden: &Schema) { + for (schema_field, golden_field) in schema.fields.iter().zip(golden.fields.iter()) { + assert!( + schema_field.name() == golden_field.name(), + "Field names don't match" + ); + assert!( + schema_field.dict_id() == golden_field.dict_id(), + "Field dict_id doesn't match" + ); + assert!( + schema_field.dict_is_ordered() == golden_field.dict_is_ordered(), + "Field dict_is_ordered doesn't match" + ); + } +} + +// some things are equivalent, but don't show up as equivalent for `==`, so we normalize here +fn normalize_col(col: Arc) -> Arc { + if let DataType::Timestamp(unit, Some(zone)) = col.data_type() { + if **zone == *"+00:00" { + arrow_cast::cast::cast(&col, &DataType::Timestamp(*unit, Some("UTC".into()))) + .expect("Could not cast to UTC") + } else { + col + } + } else { + col + } +} + +fn assert_columns_match(actual: &[Arc], expected: &[Arc]) { + for (actual, expected) in actual.iter().zip(expected) { + let actual = normalize_col(actual.clone()); + let expected = normalize_col(expected.clone()); + // note that array equality includes data_type equality + // See: https://arrow.apache.org/rust/arrow_data/equal/fn.equal.html + assert_eq!( + &actual, &expected, + "Column data didn't match. Got {actual:?}, expected {expected:?}" + ); + } +} + +pub async fn assert_scan_data( + all_data: Vec, + test_case: &TestCaseInfo, +) -> TestResult<()> { + let all_data = concat_batches(&all_data[0].schema(), all_data.iter()).unwrap(); + let all_data = sort_record_batch(all_data)?; + + let golden = read_golden(test_case.root_dir(), None).await?; + let golden = sort_record_batch(golden)?; + + assert_columns_match(all_data.columns(), golden.columns()); + assert_schema_fields_match(all_data.schema().as_ref(), golden.schema().as_ref()); + assert!( + all_data.num_rows() == golden.num_rows(), + "Didn't have same number of rows" + ); + + Ok(()) +} diff --git a/crates/test/src/acceptance/meta.rs b/crates/test/src/acceptance/meta.rs new file mode 100644 index 0000000000..6a44f2cb69 --- /dev/null +++ b/crates/test/src/acceptance/meta.rs @@ -0,0 +1,73 @@ +use std::collections::HashMap; +use std::fs::File; +use std::path::{Path, PathBuf}; + +use delta_kernel::{Error, Version}; +use serde::{Deserialize, Serialize}; +use url::Url; + +#[derive(Debug, thiserror::Error)] +pub enum AssertionError { + #[error("Invalid test case data")] + InvalidTestCase, + + #[error("Kernel error: {0}")] + KernelError(#[from] Error), +} + +pub type TestResult = std::result::Result; + +#[derive(Serialize, Deserialize, PartialEq, Eq, Debug)] +struct TestCaseInfoJson { + name: String, + description: String, +} + +#[derive(PartialEq, Eq, Debug)] +pub struct TestCaseInfo { + name: String, + description: String, + root_dir: PathBuf, +} + +impl TestCaseInfo { + /// Root path for this test cases Delta table. + pub fn table_root(&self) -> TestResult { + let table_root = self.root_dir.join("delta"); + Url::from_directory_path(table_root).map_err(|_| AssertionError::InvalidTestCase) + } + + pub fn root_dir(&self) -> &PathBuf { + &self.root_dir + } + + pub fn table_summary(&self) -> TestResult { + let info_path = self + .root_dir() + .join("expected/latest/table_version_metadata.json"); + let file = File::open(info_path).map_err(|_| AssertionError::InvalidTestCase)?; + let info: TableVersionMetaData = + serde_json::from_reader(file).map_err(|_| AssertionError::InvalidTestCase)?; + Ok(info) + } +} + +#[derive(Serialize, Deserialize, PartialEq, Eq, Debug)] +pub struct TableVersionMetaData { + pub version: Version, + pub properties: HashMap, + pub min_reader_version: i32, + pub min_writer_version: i32, +} + +pub fn read_dat_case(case_root: impl AsRef) -> TestResult { + let info_path = case_root.as_ref().join("test_case_info.json"); + let file = File::open(info_path).map_err(|_| AssertionError::InvalidTestCase)?; + let info: TestCaseInfoJson = + serde_json::from_reader(file).map_err(|_| AssertionError::InvalidTestCase)?; + Ok(TestCaseInfo { + root_dir: case_root.as_ref().into(), + name: info.name, + description: info.description, + }) +} diff --git a/crates/test/src/acceptance/mod.rs b/crates/test/src/acceptance/mod.rs new file mode 100644 index 0000000000..521fd294ae --- /dev/null +++ b/crates/test/src/acceptance/mod.rs @@ -0,0 +1,5 @@ +pub mod data; +pub mod meta; + +pub use data::*; +pub use meta::*; diff --git a/crates/test/src/concurrent.rs b/crates/test/src/concurrent.rs index aed4576925..eb5b3e6d92 100644 --- a/crates/test/src/concurrent.rs +++ b/crates/test/src/concurrent.rs @@ -3,8 +3,8 @@ use std::future::Future; use std::iter::FromIterator; use std::time::Duration; +use deltalake_core::kernel::transaction::CommitBuilder; use deltalake_core::kernel::{Action, Add, DataType, PrimitiveType, StructField, StructType}; -use deltalake_core::operations::transaction::CommitBuilder; use deltalake_core::operations::DeltaOps; use deltalake_core::protocol::{DeltaOperation, SaveMode}; use deltalake_core::{DeltaTable, DeltaTableBuilder}; @@ -37,7 +37,7 @@ async fn prepare_table( .with_columns(schema.fields().cloned()) .await?; - assert_eq!(0, table.version()); + assert_eq!(Some(0), table.version()); assert_eq!(1, table.protocol()?.min_reader_version); assert_eq!(2, table.protocol()?.min_writer_version); // assert_eq!(0, table.get_files_iter().count()); @@ -130,7 +130,6 @@ impl Worker { modification_time: 1564524294000, data_change: true, stats: None, - stats_parsed: None, tags: None, deletion_vector: None, base_row_id: None, diff --git a/crates/test/src/datafusion.rs b/crates/test/src/datafusion.rs deleted file mode 100644 index 602c115bd6..0000000000 --- a/crates/test/src/datafusion.rs +++ /dev/null @@ -1,12 +0,0 @@ -use deltalake_core::datafusion::execution::context::SessionContext; -use deltalake_core::datafusion::execution::session_state::SessionStateBuilder; -use deltalake_core::delta_datafusion::DeltaTableFactory; -use std::sync::Arc; - -pub fn context_with_delta_table_factory() -> SessionContext { - let mut state = SessionStateBuilder::new().build(); - state - .table_factories_mut() - .insert("DELTATABLE".to_string(), Arc::new(DeltaTableFactory {})); - SessionContext::new_with_state(state) -} diff --git a/crates/test/src/lib.rs b/crates/test/src/lib.rs index dd8c2a2951..34fcc70471 100644 --- a/crates/test/src/lib.rs +++ b/crates/test/src/lib.rs @@ -4,20 +4,19 @@ use std::collections::HashMap; use std::sync::Arc; use bytes::Bytes; +use deltalake_core::kernel::transaction::CommitBuilder; use deltalake_core::kernel::{Action, Add, Remove, StructType}; use deltalake_core::logstore::LogStore; use deltalake_core::operations::create::CreateBuilder; -use deltalake_core::operations::transaction::CommitBuilder; use deltalake_core::protocol::{DeltaOperation, SaveMode}; use deltalake_core::DeltaTable; use deltalake_core::DeltaTableBuilder; use deltalake_core::{ObjectStore, Path}; use tempfile::TempDir; +pub mod acceptance; pub mod clock; pub mod concurrent; -#[cfg(feature = "datafusion")] -pub mod datafusion; pub mod read; pub mod utils; @@ -114,7 +113,7 @@ pub async fn add_file( create_time: i64, commit_to_log: bool, ) { - let backend = table.object_store(); + let backend = table.log_store().object_store(None); backend.put(path, data.clone().into()).await.unwrap(); if commit_to_log { @@ -130,7 +129,6 @@ pub async fn add_file( partition_values: part_values, data_change: true, stats: None, - stats_parsed: None, tags: None, default_row_commit_version: None, base_row_id: None, diff --git a/crates/test/src/read.rs b/crates/test/src/read.rs index e6f8b6f7eb..0d62ea6cef 100644 --- a/crates/test/src/read.rs +++ b/crates/test/src/read.rs @@ -40,7 +40,7 @@ async fn read_simple_table(integration: &IntegrationContext) -> TestResult { .load() .await?; - assert_eq!(table.version(), 4); + assert_eq!(table.version(), Some(4)); assert_eq!(table.protocol()?.min_writer_version, 2); assert_eq!(table.protocol()?.min_reader_version, 1); assert_eq!( @@ -55,7 +55,7 @@ async fn read_simple_table(integration: &IntegrationContext) -> TestResult { ); let tombstones = table .snapshot()? - .all_tombstones(table.object_store().clone()) + .all_tombstones(&table.log_store()) .await? .collect::>(); assert_eq!(tombstones.len(), 31); @@ -84,7 +84,7 @@ async fn read_simple_table_with_version(integration: &IntegrationContext) -> Tes .load() .await?; - assert_eq!(table.version(), 3); + assert_eq!(table.version(), Some(3)); assert_eq!(table.protocol()?.min_writer_version, 2); assert_eq!(table.protocol()?.min_reader_version, 1); assert_eq!( @@ -100,7 +100,7 @@ async fn read_simple_table_with_version(integration: &IntegrationContext) -> Tes ); let tombstones = table .snapshot()? - .all_tombstones(table.object_store().clone()) + .all_tombstones(&table.log_store()) .await? .collect::>(); assert_eq!(tombstones.len(), 29); @@ -129,7 +129,7 @@ pub async fn read_golden(integration: &IntegrationContext) -> TestResult { .await .unwrap(); - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.protocol()?.min_writer_version, 2); assert_eq!(table.protocol()?.min_reader_version, 1); @@ -165,7 +165,7 @@ async fn read_encoded_table(integration: &IntegrationContext, root_path: &str) - .load() .await?; - assert_eq!(table.version(), 0); + assert_eq!(table.version(), Some(0)); assert_eq!(table.get_files_iter()?.count(), 2); Ok(()) diff --git a/crates/test/src/utils.rs b/crates/test/src/utils.rs index 9d52e4ea9c..7150aa5071 100644 --- a/crates/test/src/utils.rs +++ b/crates/test/src/utils.rs @@ -1,12 +1,13 @@ #![allow(dead_code, missing_docs)] -use deltalake_core::storage::ObjectStoreRef; +use deltalake_core::logstore::ObjectStoreRef; use deltalake_core::{DeltaResult, DeltaTableBuilder}; use fs_extra::dir::{copy, CopyOptions}; use std::collections::HashMap; -use std::env; use std::process::ExitStatus; use tempfile::{tempdir, TempDir}; +pub use deltalake_core::test_utils::TestTables; + pub type TestResult = Result>; pub trait StorageIntegration { @@ -128,7 +129,7 @@ impl IntegrationContext { name: impl AsRef, ) -> TestResult { self.integration - .copy_directory(&table.as_path(), name.as_ref())?; + .copy_directory(table.as_path().to_str().unwrap(), name.as_ref())?; Ok(()) } @@ -145,82 +146,6 @@ impl IntegrationContext { } } -/// Reference tables from the test data folder -pub enum TestTables { - Simple, - SimpleWithCheckpoint, - SimpleCommit, - Golden, - Delta0_8_0Partitioned, - Delta0_8_0SpecialPartitioned, - Checkpoints, - LatestNotCheckpointed, - WithDvSmall, - Custom(String), -} - -impl TestTables { - fn as_path(&self) -> String { - // env "CARGO_MANIFEST_DIR" is "the directory containing the manifest of your package", - // set by `cargo run` or `cargo test`, see: - // https://doc.rust-lang.org/cargo/reference/environment-variables.html - let dir = env!("CARGO_MANIFEST_DIR"); - let data_path = std::path::Path::new(dir).join("tests/data"); - match self { - Self::Simple => data_path.join("simple_table").to_str().unwrap().to_owned(), - Self::SimpleWithCheckpoint => data_path - .join("simple_table_with_checkpoint") - .to_str() - .unwrap() - .to_owned(), - Self::SimpleCommit => data_path.join("simple_commit").to_str().unwrap().to_owned(), - Self::Golden => data_path - .join("golden/data-reader-array-primitives") - .to_str() - .unwrap() - .to_owned(), - Self::Delta0_8_0Partitioned => data_path - .join("delta-0.8.0-partitioned") - .to_str() - .unwrap() - .to_owned(), - Self::Delta0_8_0SpecialPartitioned => data_path - .join("delta-0.8.0-special-partition") - .to_str() - .unwrap() - .to_owned(), - Self::Checkpoints => data_path.join("checkpoints").to_str().unwrap().to_owned(), - Self::LatestNotCheckpointed => data_path - .join("latest_not_checkpointed") - .to_str() - .unwrap() - .to_owned(), - Self::WithDvSmall => data_path - .join("table-with-dv-small") - .to_str() - .unwrap() - .to_owned(), - // the data path for upload does not apply to custom tables. - Self::Custom(_) => todo!(), - } - } - - pub fn as_name(&self) -> String { - match self { - Self::Simple => "simple".into(), - Self::SimpleWithCheckpoint => "simple_table_with_checkpoint".into(), - Self::SimpleCommit => "simple_commit".into(), - Self::Golden => "golden".into(), - Self::Delta0_8_0Partitioned => "delta-0.8.0-partitioned".into(), - Self::Delta0_8_0SpecialPartitioned => "delta-0.8.0-special-partition".into(), - Self::Checkpoints => "checkpoints".into(), - Self::LatestNotCheckpointed => "latest_not_checkpointed".into(), - Self::WithDvSmall => "table-with-dv-small".into(), - Self::Custom(name) => name.to_owned(), - } - } -} - /// Set environment variable if it is not set pub fn set_env_if_not_set(key: impl AsRef, value: impl AsRef) { if std::env::var(key.as_ref()).is_err() { diff --git a/crates/test/tests/data/simple_table/_delta_log/.tmp/00000000000000000005.json b/crates/test/tests/data/simple_table/_delta_log/.tmp/00000000000000000005.json new file mode 100644 index 0000000000..b24b1f9026 --- /dev/null +++ b/crates/test/tests/data/simple_table/_delta_log/.tmp/00000000000000000005.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1587968626637,"operation":"WRITE","operationParameters":{"mode":"Overwrite","partitionBy":"[]"},"readVersion":4,"isBlindAppend":true}} +{"add":{"path":"part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c001.snappy.parquet","partitionValues":{},"size":262,"modificationTime":1587968626600,"dataChange":true}} diff --git a/crates/test/tests/data/simple_table/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c001.snappy.parquet b/crates/test/tests/data/simple_table/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c001.snappy.parquet new file mode 100644 index 0000000000..3706170963 Binary files /dev/null and b/crates/test/tests/data/simple_table/part-00000-c1777d7d-89d9-4790-b38a-6ee7e24456b1-c001.snappy.parquet differ diff --git a/dev/publish.sh b/dev/publish.sh index 407ee08919..b17f6204b8 100755 --- a/dev/publish.sh +++ b/dev/publish.sh @@ -2,7 +2,7 @@ set -xe -for crate in "core" "mount" "catalog-glue" "catalog-unity" "hdfs" "lakefs" "azure" "aws" "gcp" "deltalake"; do +for crate in "derive" "core" "mount" "catalog-glue" "aws" "azure" "gcp" "catalog-unity" "hdfs" "lakefs" "deltalake"; do echo ">> Dry-run publishing ${crate}" (cd crates/${crate} && \ cargo publish \ diff --git a/dev/release/update_change_log.sh b/dev/release/update_change_log.sh index 5b4d1cf547..4a3d6687ff 100755 --- a/dev/release/update_change_log.sh +++ b/dev/release/update_change_log.sh @@ -32,12 +32,12 @@ SOURCE_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)" SOURCE_TOP_DIR="$(cd "${SOURCE_DIR}/../../" && pwd)" OUTPUT_PATH="${SOURCE_TOP_DIR}/CHANGELOG.md" -HISTORIAL_PATH="${SOURCE_TOP_DIR}/CHANGELOG-old.md" +HISTORICAL_PATH="${SOURCE_TOP_DIR}/CHANGELOG-old.md" -cp $OUTPUT_PATH $HISTORIAL_PATH +cp $OUTPUT_PATH $HISTORICAL_PATH # Remove header from historical change logs; will add back at the end. -sed -i.bak '1d' "${HISTORIAL_PATH}" +sed -i.bak '1d' "${HISTORICAL_PATH}" # use exclude-tags-regex to filter out tags used in the wrong language pushd "${SOURCE_TOP_DIR}" @@ -59,7 +59,7 @@ LINE_COUNT=$(wc -l <"${OUTPUT_PATH}") sed -i.bak "$(( $LINE_COUNT-3 )),$ d" "${OUTPUT_PATH}" # Add historical change log back in -cat $HISTORIAL_PATH >> $OUTPUT_PATH +cat $HISTORICAL_PATH >> $OUTPUT_PATH # Remove temporary files -rm $HISTORIAL_PATH +rm $HISTORICAL_PATH diff --git a/docs/how-delta-lake-works/architecture-of-delta-table.md b/docs/how-delta-lake-works/architecture-of-delta-table.md index b2426d75d5..a783a88d7a 100644 --- a/docs/how-delta-lake-works/architecture-of-delta-table.md +++ b/docs/how-delta-lake-works/architecture-of-delta-table.md @@ -194,7 +194,7 @@ Here are the contents of the `_delta_log/0002.json` file: } ``` -This transaction adds a data file and marks the two exising data files for removal. Marking a file for removal in the transaction log is known as "tombstoning the file" or a "logical delete". This is different from a "physical delete" which actually removes the data file from storage. +This transaction adds a data file and marks the two existing data files for removal. Marking a file for removal in the transaction log is known as "tombstoning the file" or a "logical delete". This is different from a "physical delete" which actually removes the data file from storage. ## How Delta table operations differ from data lakes diff --git a/docs/integrations/delta-lake-daft.md b/docs/integrations/delta-lake-daft.md index 9722873648..f48058ce0f 100644 --- a/docs/integrations/delta-lake-daft.md +++ b/docs/integrations/delta-lake-daft.md @@ -85,7 +85,7 @@ You can use `write_deltalake` to write a Daft DataFrame to a Delta table: df.write_deltalake("tmp/daft-table", mode="overwrite") ``` -Daft supports multiple write modes. See the [Daft documentation](https://www.getdaft.io/projects/docs/en/latest/api_docs/doc_gen/dataframe_methods/daft.DataFrame.write_deltalake.html#daft.DataFrame.write_deltalake) for more information. +Daft supports multiple write modes. See the [Daft documentation](https://www.getdaft.io/projects/docs/en/stable/api/dataframe/?h=write+delta#daft.DataFrame.write_deltalake) for more information. ## What can I do with a Daft DataFrame? @@ -198,7 +198,7 @@ Read [High-Performance Querying on Massive Delta Lake Tables with Daft](https:// Daft has a rich multimodal type-system with support for Python objects, Images, URLs, Tensors and more. -The [Expressions API](https://www.getdaft.io/projects/docs/en/latest/api_docs/expressions.html) provides useful tools to work with these data types. By combining multimodal data support with the [User-Defined Functions API](https://www.getdaft.io/projects/docs/en/latest/api_docs/udf.html) you can run ML workloads right within your DataFrame. +The [Expressions API](https://www.getdaft.io/projects/docs/en/stable/api/expressions) provides useful tools to work with these data types. By combining multimodal data support with the [User-Defined Functions API](https://www.getdaft.io/projects/docs/en/stable/api/udf) you can run ML workloads right within your DataFrame. Take a look at the notebook in the [`delta-examples` Github repository](https://github.com/delta-io/delta-examples) for a closer look at how Daft handles URLs, images and ML applications. @@ -206,4 +206,4 @@ Take a look at the notebook in the [`delta-examples` Github repository](https:// Excited about Daft and want to contribute? Join them on [Github](https://github.com/Eventual-Inc/Daft) 🚀 -Like many technologies, Daft collects some non-identifiable telemetry to improve the product. This is stricly non-identifiable metadata. You can disable telemetry by setting the following environment variable: `DAFT_ANALYTICS_ENABLED=0`. Read more in the [Daft documentation](https://www.getdaft.io/projects/docs/en/stable/resources/telemetry/). +Like many technologies, Daft collects some non-identifiable telemetry to improve the product. This is strictly non-identifiable metadata. You can disable telemetry by setting the following environment variable: `DAFT_ANALYTICS_ENABLED=0`. Read more in the [Daft documentation](https://www.getdaft.io/projects/docs/en/stable/resources/telemetry/). diff --git a/docs/integrations/delta-lake-dagster.md b/docs/integrations/delta-lake-dagster.md index 3aa5a505a6..b96aa6b496 100644 --- a/docs/integrations/delta-lake-dagster.md +++ b/docs/integrations/delta-lake-dagster.md @@ -5,15 +5,16 @@ Delta Lake is a great storage format for Dagster workflows. This page will expla You will learn how to use the Delta Lake I/O Manager to read and write your Dagster Software-Defined Assets (SDAs). You will also learn about the unique advantages Delta Lake offers the Dagster community. Here are some of the benefits that Delta Lake provides Dagster users: + - native PyArrow integration for lazy computation of large datasets, - more efficient querying with file skipping via Z Ordering and liquid clustering - built-in vacuuming to remove unnecessary files and versions - ACID transactions for reliable writes - smooth versioning integration so that versions can be use to trigger downstream updates. -- surfacing table stats based on the file statistics - +- surfacing table stats based on the file statistics ## Dagster I/O Managers + Dagster uses [I/O Managers](https://docs.dagster.io/concepts/io-management/io-managers#overview) to simplify data reads and writes. I/O Managers help you reduce boilerplate code by storing Dagster Asset and Op outputs and loading them as inputs to downstream objects. They make it easy to change where and how your data is stored. You only need to define your I/O Manager and its settings (such as storage location and schema) once and the I/O Manager will take care of correctly reading and writing all your Dagster Assets automatically. @@ -21,7 +22,8 @@ You only need to define your I/O Manager and its settings (such as storage locat If you need lower-level access than the Dagster I/O Managers provide, take a look at the Delta Table Resource. ## The Delta Lake I/O Manager -You can easily read and write Delta Lake Tables from Dagster by using the `DeltaLakeIOManager()`. + +You can easily read and write Delta Lake Tables from Dagster by using the `DeltaLakeIOManager()`. Install the DeltaLakeIOManager: @@ -30,9 +32,11 @@ pip install dagster-deltalake ``` Next, configure the following settings in your project’s `__init__.py` file: + - `io_manager`: set this to `DeltaLakeIOManager()`, this sets the default I/O Manager for all your Assets Within the DeltaLakeIOManager, define: + - `root_uri`: the root path where your Delta Tables will be created - `storage_options`: configuration for accessing storage location - `schema`: name of schema to use (optional, defaults to public) @@ -52,9 +56,10 @@ defs = Definitions( Now, when you materialize an Asset, it will be saved as a Delta Lake in a folder `dagster_deltalake/asset_name` under the root directory `path/to/deltalake`. -The default Delta Lake I/O Manager supports Arrow reads and writes. You can also use the Delta Lake I/O Manager with [pandas](#using-delta-lake-and-dagster-with-pandas) or [polars](#using-delta-lake-and-dagster-with-polars). +The default Delta Lake I/O Manager supports Arrow reads and writes. You can also use the Delta Lake I/O Manager with [pandas](#using-delta-lake-and-dagster-with-pandas) or [polars](#using-delta-lake-and-dagster-with-polars). ## Creating Delta Lake Tables with Dagster + You don’t need to do anything else to store your Dagster Assets as Delta Lake tables. The I/O Manager will handle storing and loading your Assets as Delta Lake tables from now on. You can proceed to write Dagster code as you normally would. For example, you can create an Asset that reads in some toy data about animals and writes it out to an Arrow Table: @@ -77,9 +82,11 @@ def raw_dataset() -> pa.Table: When you materialize the Asset defined above (using the config settings defined earlier), the Delta Lake I/O Manager will create the table `dagster_deltalake/iris_dataset` if it doesn’t exist yet. ### Overwrites when Rematerializing Assets + If the table does already exist at the specified location, the Delta Lake I/O Manager will perform an overwrite. Delta Lake’s transaction log maintains a record of all changes to your Delta Lake tables. You can inspect the record of changes to your Delta Lake tables by taking a look at these transaction logs. ## Loading Delta Lake Tables in Downstream Assets + You can use Assets stored as Delta Lake tables as input to downstream Assets. Dagster and the Delta Lake I/O Manager make this easy for you. You can write Dagster code as you normally would. Pass the upstream Asset as an argument to the downstream object to set up the dependency. Make sure to define the correct data type. @@ -98,7 +105,8 @@ def clean_dataset(raw_dataset: pa.Table) -> pa.Table: ``` ## Reading Existing Delta Lake Tables into Dagster -You can make existing Delta Lake tables (that were not created in Dagster) available to your Dagster assets. Use the `SourceAsset` object and pass the table name as the key argument: + +You can make existing Delta Lake tables (that were not created in Dagster) available to your Dagster assets. Use the `SourceAsset` object and pass the table name as the key argument: ``` from dagster import SourceAsset @@ -109,7 +117,8 @@ iris_harvest_data = SourceAsset(key="more_animal_data") This will load a table `more_animal_data` located at `/` as configured in the Definitions object above (see [Delta Lake I/O Manager](#the-delta-lake-io-manager) section). ## Column Pruning -You can often improve the efficiency of your computations by only loading specific columns of your Delta table. This is called column pruning. + +You can often improve the efficiency of your computations by only loading specific columns of your Delta table. This is called column pruning. With the Delta Lake I/O manager, you can select specific columns to load defining the `columns` in the `metadata` parameter of the `AssetIn` that loads the upstream Asset: @@ -137,6 +146,7 @@ def mammal_data(mammal_bool: pa.Table) -> pa.Table: Here, we select only the `sepal_length_cm` and `sepal_width_cm` columns from the `iris_dataset` table and load them into an `AssetIn` object called `iris_sepal`. This AssetIn object is used to create a new Asset `sepal_data`, containing only the selected columns. ## Working with Partitioned Assets + Partitioning is an important feature of Delta Lake that can make your computations more efficient. The Delta Lake I/O manager helps you read and write partitioned data easily. You can work with static partitions, time-based partitions, multi-partitions, and dynamic partitions. For example, you can partition the Iris dataset on the `species` column as follows: @@ -165,7 +175,8 @@ def dataset_partitioned( To partition your data, make sure to include the relevant `partitions_def` and `metadata` arguments to the `@asset` decorator. Refer to the Dagster documentation on [partitioning assets](https://docs.dagster.io/concepts/partitions-schedules-sensors/partitioning-assets) for more information. ## Using Delta Lake and Dagster with Pandas -To read and write data to Delta Lake using pandas, use the `DeltaLakePandasIOManager()`. + +To read and write data to Delta Lake using pandas, use the `DeltaLakePandasIOManager()`. You will need to install it using: @@ -212,7 +223,8 @@ def iris_dataset() -> pd.DataFrame: ``` ## Using Delta Lake and Dagster with Polars -To read and write data to Delta Lake using polars, use the `DeltaLakePolarsIOManager()`. + +To read and write data to Delta Lake using polars, use the `DeltaLakePolarsIOManager()`. You will need to install it using: @@ -260,19 +272,23 @@ def iris_dataset() -> pl.DataFrame: ``` ## Delta Lake Table Resource + I/O managers are a helpful tool in many common usage situations. But when you need lower-level access, the I/O Manager might not be the right tool to use. In these cases you may want to use the Delta Lake Table Resource. The Delta Lake Table Resource is a low-level access method to the table object. It gives you more fine-grained control and allows for modeling of more complex data. You can also use the Table Resource to run optimization and vacuuming jobs. ## Schema and Constraint Enforcement -Delta Lake provides built-in checks to ensure schema consistency when appending data to a table, as well as the ability to evolve the schema. This is a great feature for the Dagster community as it prevents bad data from being appended to tables, ensuring data consistency and accuracy. + +Delta Lake provides built-in checks to ensure schema consistency when appending data to a table, as well as the ability to evolve the schema. This is a great feature for the Dagster community as it prevents bad data from being appended to tables, ensuring data consistency and accuracy. Read more about how to add constraints to a table in [the Delta Lake documentation](https://delta-io.github.io/delta-rs/usage/constraints/). ## Z-Ordering + Delta Lake offers Z-ordering functionality to colocate similar data in the same files. This can make your Delta Table queries much more efficient via file skipping. Dagster users can now benefit from this great feature through the Delta Lake I/O Manager. Read more about Z-Ordering on [the Delta Lake blog](https://delta.io/blog/2023-06-03-delta-lake-z-order/). ## Contribute + To contribute to the Delta Lake and Dagster integration, go to [link] diff --git a/docs/upgrade-guides/guide-1.0.0.md b/docs/upgrade-guides/guide-1.0.0.md new file mode 100644 index 0000000000..2391d0e78e --- /dev/null +++ b/docs/upgrade-guides/guide-1.0.0.md @@ -0,0 +1,220 @@ +# DeltaLake 1.0.0 Migration Guide + +DeltaLake 1.0.0 introduces significant changes, including the removal of the legacy Python writer engine. All write operations are now delegated to the `Rust` engine, which provides enhanced performance via `streaming execution` and `multipart uploads` for improved concurrency and reduced memory usage. + +In addition, DeltaLake no longer has a hard dependency on PyArrow, making it more lightweight and suitable for use in minimal or size-constrained environments. Arrow data handling between Python and Rust is now powered by `arro3`, a lightweight Arrow implementation backed by `arrow-rs`. + +For users who still require PyArrow-based read functionality, it remains available as an optional dependency. To continue using these features, install DeltaLake with the `pyarrow` extra, `deltalake[pyarrow]`. + +## Breaking changes + +### `arro3` adoption + +DeltaLake 1.0.0 introduces support for Arrow PyCapsule protocol using a lightweight arrow implementation with `arro3`. As a result, the `write_deltalake` and `DeltaTable.merge` functions now only accept inputs that implement one of the following interfaces: + +- `ArrowStreamExportable` +- `ArrowArrayExportable` + +This means you can directly pass data from compatible libraries, such as: + +- Polars DataFrame +- PyArrow Table or RecordBatchReader +- Pandas DataFrame (via its Arrow export support) + +DeltaLake will consume these objects efficiently through the Arrow C Data Interface. + +#### Schema, Types and Fields + +Schema handling has also transitioned from PyArrow to `arro3`. Internally, schema, types, and fields are now defined and interpreted via the Arrow C Data Interface: + +Our schema code also no longer converts to pyarrow but converts to `arro3` schema/types/fields directly over Arrow C Data Interface. + +- `to_pyarrow` → replaced with `to_arrow` +- `from_pyarrow` -> replaced with `from_arrow` (can accept any object implementing `ArrowSchemaExportable`) + +All DeltaLake schema components—such as schema, field, and type objects—now implement the `__arrow_c_schema__` protocol. This means they are directly consumable by any library that supports the Arrow C Data Interface (e.g., Polars, PyArrow, pandas). + +#### Arrow Object Outputs + +Several DeltaLake APIs now return arro3-based Arrow objects instead of PyArrow objects. This includes: + +- `DeltaTable.get_add_actions` +- `DeltaTable.load_cdf` +- `QueryBuilder.execute` + +These returned objects are fully compatible with libraries like Polars and PyArrow. For example: + +```python +import polars as pl +import pyarrow as pa + +dt = DeltaTable("test") +rb = dt.load_df() + +# With polars +df = pl.DataFrame(rb) + +# With pyarrow +tbl = pa.table(df) +``` + +#### `create` API + +The `create` API now accepts schema definitions as either: + +- A `DeltaSchema` object, or +- Any object implementing the `ArrowSchemaExportable` protocol (e.g., a PyArrow schema) + +This ensures full interoperability with modern Arrow-compatible libraries while leveraging the performance and flexibility of the Arrow C Data Interface. + + +### `write_deltalake` API + +#### `engine` parameter removed + +The engine parameter has been removed. All writes now default to the Rust engine. + +Before: + +```python +write_deltalake( + "mytable", + data=data, + mode="append", + engine="rust", +) +``` + +After: + +```python +write_deltalake( + "mytable", + data=data, + mode="append", +) +``` + +### PyArrow-specific parameters removed + +The following parameters related to the deprecated Python engine are no longer supported: + +- `file_options: Optional[ds.ParquetFileWriteOptions]` +- `max_partitions: Optional[int]` +- `max_open_files: int` +- `max_rows_per_file: int` +- `min_rows_per_group: int` +- `max_rows_per_group: int` + +#### `partition_filters` removed + +The partition_filters parameter has been removed. Use the `predicate` parameter as an alternative for overwriting a subset during writes + +#### `large_dtypes` removed + +The `large_dtypes` parameter has been removed. DeltaLake now always passes through all Arrow data types without modification for both `write_deltalake` and `DeltaTable.merge`. + +### QueryBuilder + +QueryBuilder no longer implements `show`, and `execute` will directly return a RecordBatchReader. Additionally the experimental flag has been removed. + +### CommitProperties + +#### `custom_metadata` replaced with `commit_properties` + +The custom_metadata argument has been replaced by the commit_properties parameter on the following APIs: + +- convert_to_deltalake +- DeltaTable + - create + - vacuum + - update + - merge + - restore + - repair + - add_columns + - add_constraint + - set_table_properties + - compact + - z_order + - delete + +Before: + +```python +convert_to_deltalake( + uri="mytable", + custom_metadata={"foo":"bar"}, +) +``` + +After: + +```python +convert_to_deltalake( + uri="mytable", + commit_properties=CommitProperties(custom_metadata={"foo":"bar"}), +) +``` + +### DeltaTable + +#### removed `from_data_catalog` + +This method was previously unimplemented and has now been fully removed from the DeltaTable class. + +#### removed `get_earliest_version` + +This method is potentially very expensive while we could not find a clear use case for it. +If you have a specific use case, please open an issue on GitHub. + +#### `transaction_versions` changed to `transaction_version` + +`transaction_versions` has been renamed to `transaction_version` and now returns a single version +number for a specific application instead of a dictionary for all applications. This allows +for internal optimisations and aligns more with the use case for idempotent writes. + +To store more complex state as part of the delta log, use `domainMetadata`. + +#### removed `get_num_index_cols`, `get_stats_columns` and `check_can_write_timestamp_ntz`. + +These methods were rxposed for use by the `pyarrow` engine, which no longer exists. + +#### deprecated `files` + +The `files` method has been deprecated and will be removed in a future release. Use `file_uris` instead. +The relative paths reported by `files` cannot properly be interpreted when using features like shallow clones. +`file_uris` reposrts absolute paths that can handle a more dynamic path resolution. + +### Internal changes + +#### `WriterProperties`, `ColumnProperties` and `BloomFilterProperties` moved to `deltalake.writer.properties` + +Can be imported directly with: + +```python +from deltalake import WriterProperties, ColumnProperties, BloomFilterProperties +``` + +#### `AddAction`, `CommitProperties` and `PostCommithookProperties` moved to `deltalake.transaction` + +Can be imported directly with: + +```python +from deltalake import AddAction, CommitProperties, PostCommithookProperties +``` + +### New features + +#### Public APIs for transaction management + +Functionality previously limited to internal PyArrow writer methods is now publicly available: + +- `write_new_deltalake` has been renamed to `create_table_with_add_actions` and is now exposed under deltalake.transaction. +- You can now initiate a write transaction on an existing table using: `DeltaTable.create_write_transaction(...)` + +### Behavior changes with unsupported delta features ⚠️ + +Previously, it was possible to work with Delta tables that included unsupported features (e.g., Deletion Vectors) as long as those features were not actively used during an operation/interaction. + +As of version 1.0.0, this is no longer allowed. DeltaLake will now raise an error if it detects unsupported features in a table. This change ensures safety and avoids inconsistencies stemming from partial or undefined behavior. diff --git a/docs/usage/appending-overwriting-delta-lake-table.md b/docs/usage/appending-overwriting-delta-lake-table.md index 68d00cdfcb..fe2002f2ab 100644 --- a/docs/usage/appending-overwriting-delta-lake-table.md +++ b/docs/usage/appending-overwriting-delta-lake-table.md @@ -1,6 +1,6 @@ # Appending to and overwriting a Delta Lake table -This section explains how to append to an exising Delta table and how to overwrite a Delta table. +This section explains how to append to an existing Delta table and how to overwrite a Delta table. ## Delta Lake append transactions @@ -28,7 +28,7 @@ Append two additional rows of data to the table: ``` === "Rust" - ```rust +`rust let table = open_table("tmp/some-table").await?; DeltaOps(table).write(RecordBatch::try_new( Arc::new(Schema::new(vec![ @@ -41,7 +41,7 @@ Append two additional rows of data to the table: "dd", "ee" ])), ])).with_save_mode(SaveMode::Append).await?; - ``` + ` Here are the updated contents of the Delta table: @@ -61,15 +61,15 @@ Now let's see how to perform an overwrite transaction. ## Delta Lake overwrite transactions -Now let's see how to overwrite the exisitng Delta table. +Now let's see how to overwrite the existing Delta table. === "Python" - ```python +`python df = pd.DataFrame({"num": [11, 22], "letter": ["aa", "bb"]}) write_deltalake("tmp/some-table", df, mode="overwrite") - ``` + ` === "Rust" - ```rust +`rust let table = open_table("tmp/some-table").await?; DeltaOps(table).write(RecordBatch::try_new( Arc::new(Schema::new(vec![ @@ -82,7 +82,7 @@ Now let's see how to overwrite the exisitng Delta table. "a", "b", "c", ])), ])).with_save_mode(SaveMode::Overwrite).await?; - ``` + ` Here are the contents of the Delta table after the overwrite operation: ``` @@ -94,7 +94,7 @@ Here are the contents of the Delta table after the overwrite operation: +-------+----------+ ``` -Overwriting just performs a logical delete. It doesn't physically remove the previous data from storage. Time travel back to the previous version to confirm that the old version of the table is still accessable. +Overwriting just performs a logical delete. It doesn't physically remove the previous data from storage. Time travel back to the previous version to confirm that the old version of the table is still accessible. === "Python" @@ -103,11 +103,10 @@ Overwriting just performs a logical delete. It doesn't physically remove the pr ``` === "Rust" - ```rust +`rust let mut table = open_table("tmp/some-table").await?; table.load_version(1).await?; - ``` - + ` ``` +-------+----------+ diff --git a/docs/usage/examining-table.md b/docs/usage/examining-table.md index 631d8a8ef8..efc8255358 100644 --- a/docs/usage/examining-table.md +++ b/docs/usage/examining-table.md @@ -57,7 +57,7 @@ the table will be loaded into. let schema = table.get_schema()?; println!("schema: {:?}", schema); ``` -These schemas have a JSON representation that can be retrieved. +These schemas have a JSON representation that can be retrieved. === "Python" To reconstruct from json, use [DeltaTable.schema.to_json()][deltalake.schema.Schema.to_json]. @@ -73,10 +73,10 @@ These schemas have a JSON representation that can be retrieved. It is also possible to retrieve the Arrow schema: === "Python" - Use [DeltaTable.schema.to_pyarrow()][deltalake.schema.Schema.to_pyarrow] to retrieve the PyArrow schema: + Use [DeltaTable.schema.to_arrow()][deltalake.schema.Schema.to_arrow] to retrieve the Arro3 schema: ``` python - >>> dt.schema().to_pyarrow() + >>> dt.schema().to_arrow() id: int64 ``` === "Rust" @@ -129,6 +129,8 @@ which provide the list of files that are part of the table and metadata about them, such as creation time, size, and statistics. You can get a data frame of the add actions data using `DeltaTable.get_add_actions`: + + === "Python" ``` python >>> from deltalake import DeltaTable @@ -161,4 +163,6 @@ This works even with past versions of the table: table.load_version(0).await?; let actions = table.snapshot()?.add_actions_table(true)?; println!("{}", pretty_format_batches(&vec![actions])?); - ``` \ No newline at end of file + ``` + + diff --git a/docs/usage/optimize/small-file-compaction-with-optimize.md b/docs/usage/optimize/small-file-compaction-with-optimize.md index f5a6561380..b505f6235b 100644 --- a/docs/usage/optimize/small-file-compaction-with-optimize.md +++ b/docs/usage/optimize/small-file-compaction-with-optimize.md @@ -254,7 +254,8 @@ Here’s the output of the command: 'preserveInsertionOrder': True} ``` -The optimize operation has added 5 new files and marked 100 exisitng files for removal (this is also known as “tombstoning” files). It has compacted the 100 tiny files into 5 larger files. +The optimize operation has added 5 new files and marked 100 existing files for removal +(this is also known as “tombstoning” files). It has compacted the 100 tiny files into 5 larger files. Let’s append some more data to the Delta table and see how we can selectively run optimize on the new data that’s added. @@ -288,7 +289,9 @@ Let’s append another 24 hours of data to the Delta table: } ``` -We can use `get_add_actions()` to introspect the table state. We can see that `2021-01-06` has only a few hours of data so far, so we don't want to optimize that yet. But `2021-01-05` has all 24 hours of data, so it's ready to be optimized. +We can use `get_add_actions()` to introspect the table state. We can see that `2021-01-06` +has only a few hours of data so far, so we don't want to optimize that yet. But `2021-01-05` +has all 24 hours of data, so it's ready to be optimized. === "Python" ```python @@ -311,10 +314,10 @@ We can use `get_add_actions()` to introspect the table state. We can see that `2 let ctx = SessionContext::new(); ctx.register_batch("observations", batch.clone())?; let df = ctx.sql(" - SELECT \"partition.date\", - COUNT(*) - FROM observations - GROUP BY \"partition.date\" + SELECT \"partition.date\", + COUNT(*) + FROM observations + GROUP BY \"partition.date\" ORDER BY \"partition.date\"").await?; df.show().await?; @@ -331,10 +334,10 @@ We can use `get_add_actions()` to introspect the table state. We can see that `2 +----------------+----------+ ``` -To optimize a single partition, you can pass in a `partition_filters` argument speficying which partitions to optimize. +To optimize a single partition, you can pass in a `partition_filters` argument specifying which partitions to optimize. === "Python" ```python - dt.optimize(partition_filters=[("date", "=", "2021-01-05")]) + dt.optimize.compact(partition_filters=[("date", "=", "2021-01-05")]) ``` === "Rust" @@ -368,7 +371,9 @@ To optimize a single partition, you can pass in a `partition_filters` argument s 'preserveInsertionOrder': True} ``` -This optimize operation tombstones 21 small data files and adds one file with all the existing data properly condensed. Let’s take a look a portion of the `_delta_log/00000000000000000125.json` file, which is the transaction log entry that corresponds with this incremental optimize command. +This optimize operation tombstones 21 small data files and adds one file with all the existing +data properly condensed. Let’s take a look a portion of the `_delta_log/00000000000000000125.json` +file, which is the transaction log entry that corresponds with this incremental optimize command. ```python { @@ -416,9 +421,11 @@ This optimize operation tombstones 21 small data files and adds one file with al } ``` -The trasaction log indicates that many files have been tombstoned and one file is added, as expected. +The transaction log indicates that many files have been tombstoned and one file is added, as expected. -The Delta Lake optimize command “removes” data by marking the data files as removed in the transaction log. The optimize command doesn’t physically delete the Parquet file from storage. Optimize performs a “logical remove” not a “physical remove”. +The Delta Lake optimize command “removes” data by marking the data files as removed in the transaction log. +The optimize command doesn’t physically delete the Parquet file from storage. +Optimize performs a “logical remove” not a “physical remove”. Delta Lake uses logical operations so you can time travel back to earlier versions of your data. You can vacuum your Delta table to physically remove Parquet files from storage if you don’t need to time travel and don’t want to pay to store the tombstoned files. @@ -493,7 +500,7 @@ Delta tables can accumulate small files for a variety of reasons: * User error: users can accidentally write files that are too small. Users should sometimes repartition in memory before writing to disk to avoid appending files that are too small. * Frequent appends: systems that append more often tend to append more smaller files. A pipeline that appends every minute will generally generate ten times as many small files compared to a system that appends every ten minutes. -* Appending to partitioned data lakes with high cardinality columns can also cause small files. If you append every hour to a table that’s partitioned on a column with 1,000 distinct values, then every append could create 1,000 new files. Partitioning by date avoids this problem because the data isn’t split up across partitions in this manner. +* Appending to partitioned data lakes with high cardinality columns can also cause small files. If you append every hour to a table that’s partitioned on a column with 1,000 distinct values, then every append could create 1,000 new files. Partitioning by date avoids this problem because the data isn’t split up across partitions in this manner. ## Conclusion diff --git a/docs/usage/querying-delta-tables.md b/docs/usage/querying-delta-tables.md index 848f005ba4..1f1fa1f654 100644 --- a/docs/usage/querying-delta-tables.md +++ b/docs/usage/querying-delta-tables.md @@ -18,7 +18,7 @@ To load into Pandas or a PyArrow table use the `DeltaTable.to_pandas` and `Delta ``` python >>> from deltalake import DeltaTable >>> dt = DeltaTable("../rust/tests/data/delta-0.8.0-partitioned") ->>> dt.schema().to_pyarrow() +>>> dt.schema().to_arrow() value: string year: string month: string @@ -117,7 +117,7 @@ Dask Name: read-parquet, 6 tasks 1 7 2021 12 20 ``` -When working with the Rust API, Apache Datafusion can be used to query data from a delta table. +When working with the Rust API, Apache Datafusion can be used to query data from a delta table. ```rust let table = deltalake::open_table("../rust/tests/data/delta-0.8.0-partitioned").await?; @@ -134,4 +134,4 @@ let ctx = SessionContext::new(); let dataframe = ctx.read_table( Arc::new(table.clone()))?; let df = dataframe.filter(col("year").eq(lit(2021)))?.select(vec![col("value")])?; df.show().await?; -``` \ No newline at end of file +``` diff --git a/mkdocs.yml b/mkdocs.yml index a44f35846e..79b6102106 100644 --- a/mkdocs.yml +++ b/mkdocs.yml @@ -102,6 +102,8 @@ nav: - Architecture: how-delta-lake-works/architecture-of-delta-table.md - Transactions: how-delta-lake-works/delta-lake-acid-transactions.md - File skipping: how-delta-lake-works/delta-lake-file-skipping.md + - Upgrade guides: + - Version 1.0.0: upgrade-guides/guide-1.0.0.md not_in_nav: | /_build/ diff --git a/proofs/Cargo.lock b/proofs/stateright/Cargo.lock similarity index 100% rename from proofs/Cargo.lock rename to proofs/stateright/Cargo.lock diff --git a/proofs/Cargo.toml b/proofs/stateright/Cargo.toml similarity index 100% rename from proofs/Cargo.toml rename to proofs/stateright/Cargo.toml diff --git a/proofs/README.md b/proofs/stateright/README.md similarity index 100% rename from proofs/README.md rename to proofs/stateright/README.md diff --git a/proofs/src/main.rs b/proofs/stateright/src/main.rs similarity index 99% rename from proofs/src/main.rs rename to proofs/stateright/src/main.rs index 32511d6057..89d2564d16 100644 --- a/proofs/src/main.rs +++ b/proofs/stateright/src/main.rs @@ -452,7 +452,7 @@ impl Model for AtomicRenameSys { } Action::CheckSourceObjectDeleted(wid) => { let src = state.writer_ctx[wid].lock_data.src.as_str(); - // HEAD objec to check for existence + // HEAD object to check for existence if state.blob_store_deleted(src) { let mut writer = &mut state.writer_ctx[wid]; // source object cleaned by up another worker's repair, it's not a real diff --git a/tlaplus/README.md b/proofs/tlaplus/README.md similarity index 100% rename from tlaplus/README.md rename to proofs/tlaplus/README.md diff --git a/tlaplus/dynamodblock.tla b/proofs/tlaplus/dynamodblock.tla similarity index 100% rename from tlaplus/dynamodblock.tla rename to proofs/tlaplus/dynamodblock.tla diff --git a/tlaplus/dynamodblock.toolbox/.project b/proofs/tlaplus/dynamodblock.toolbox/.project similarity index 100% rename from tlaplus/dynamodblock.toolbox/.project rename to proofs/tlaplus/dynamodblock.toolbox/.project diff --git a/tlaplus/dynamodblock.toolbox/.settings/org.lamport.tla.toolbox.prefs b/proofs/tlaplus/dynamodblock.toolbox/.settings/org.lamport.tla.toolbox.prefs similarity index 100% rename from tlaplus/dynamodblock.toolbox/.settings/org.lamport.tla.toolbox.prefs rename to proofs/tlaplus/dynamodblock.toolbox/.settings/org.lamport.tla.toolbox.prefs diff --git a/tlaplus/dynamodblock.toolbox/dynamodblock___fast_dev.launch b/proofs/tlaplus/dynamodblock.toolbox/dynamodblock___fast_dev.launch similarity index 100% rename from tlaplus/dynamodblock.toolbox/dynamodblock___fast_dev.launch rename to proofs/tlaplus/dynamodblock.toolbox/dynamodblock___fast_dev.launch diff --git a/tlaplus/dynamodblock.toolbox/dynamodblock___full_tiimeless.launch b/proofs/tlaplus/dynamodblock.toolbox/dynamodblock___full_tiimeless.launch similarity index 100% rename from tlaplus/dynamodblock.toolbox/dynamodblock___full_tiimeless.launch rename to proofs/tlaplus/dynamodblock.toolbox/dynamodblock___full_tiimeless.launch diff --git a/python/Cargo.toml b/python/Cargo.toml index 5efa549b2f..b63fd187ac 100644 --- a/python/Cargo.toml +++ b/python/Cargo.toml @@ -1,7 +1,10 @@ [package] name = "deltalake-python" -version = "1.0.0-rc0" -authors = ["Qingping Hou ", "Will Jones "] +version = "1.0.2" +authors = [ + "Qingping Hou ", + "Will Jones ", +] homepage = "https://github.com/delta-io/delta-rs" license = "Apache-2.0" description = "Native Delta Lake Python binding based on delta-rs with Pandas integration" @@ -17,17 +20,24 @@ doc = false [dependencies] delta_kernel.workspace = true +pyo3-arrow = { version = "0.9.0", default-features = false } + # arrow arrow-schema = { workspace = true, features = ["serde"] } # datafusion +# datafusion-catalog = { workspace = true } +datafusion-expr = { workspace = true } datafusion-ffi = { workspace = true } +datafusion-physical-expr = { workspace = true } +datafusion-physical-plan = { workspace = true } # serde serde = { workspace = true } serde_json = { workspace = true } # "stdlib" +async-trait = { workspace = true } chrono = { workspace = true } env_logger = "0" regex = { workspace = true } @@ -44,7 +54,12 @@ tokio = { workspace = true, features = ["rt-multi-thread"] } deltalake-mount = { path = "../crates/mount" } # catalog-unity -deltalake-catalog-unity = { path = "../crates/catalog-unity", features = ["aws", "azure", "gcp", "r2"] } +deltalake-catalog-unity = { path = "../crates/catalog-unity", features = [ + "aws", + "azure", + "gcp", + "r2", +] } # Non-unix or emscripten os [target.'cfg(any(not(target_family = "unix"), target_os = "emscripten"))'.dependencies] @@ -52,20 +67,31 @@ mimalloc = { version = "0.1", default-features = false } # Unix (excluding macOS & emscripten) → jemalloc [target.'cfg(all(target_family = "unix", not(target_os = "macos"), not(target_os = "emscripten")))'.dependencies] -jemallocator = { version = "0.5", features = ["disable_initial_exec_tls", "background_threads"] } +jemallocator = { version = "0.5", features = [ + "disable_initial_exec_tls", + "background_threads", +] } # macOS → jemalloc (without background_threads) (https://github.com/jemalloc/jemalloc/issues/843) [target.'cfg(all(target_family = "unix", target_os = "macos"))'.dependencies] jemallocator = { version = "0.5", features = ["disable_initial_exec_tls"] } [dependencies.pyo3] -version = "0.23.4" +version = "0.24.0" features = ["extension-module", "abi3", "abi3-py39"] [dependencies.deltalake] path = "../crates/deltalake" version = "0" -features = ["azure", "gcs", "python", "datafusion", "unity-experimental", "hdfs", "lakefs"] +features = [ + "azure", + "gcs", + "python", + "datafusion", + "unity-experimental", + "hdfs", + "lakefs", +] [features] default = ["rustls"] diff --git a/python/Makefile b/python/Makefile index 977d19e322..fb61f2fc8f 100644 --- a/python/Makefile +++ b/python/Makefile @@ -1,23 +1,12 @@ .DEFAULT_GOAL := help -PACKAGE_VERSION := $(shell grep version Cargo.toml | head -n 1 | awk '{print $$3}' | tr -d '"' ) -DAT_VERSION := 0.0.2 +PACKAGE_VERSION := $(shell grep version Cargo.toml | head -n 1 | awk '{print $$3}' | tr -d '"' | tr -d '-' ) .PHONY: setup setup: ## Setup the requirements $(info --- Setup dependencies ---) uv sync --no-install-project --all-extras -.PHONY: setup-dat -setup-dat: ## Download DAT test files - mkdir -p dat-data - rm -rf dat-data/v$(DAT_VERSION) - curl -L --silent --output dat-data/deltalake-dat-v$(DAT_VERSION).tar.gz \ - https://github.com/delta-incubator/dat/releases/download/v$(DAT_VERSION)/deltalake-dat-v$(DAT_VERSION).tar.gz - tar --no-same-permissions -xzf dat-data/deltalake-dat-v$(DAT_VERSION).tar.gz - mv out dat-data/v$(DAT_VERSION) - rm dat-data/deltalake-dat-v$(DAT_VERSION).tar.gz - .PHONY: build build: setup ## Build Python binding of delta-rs $(info --- Build Python binding ---) @@ -78,7 +67,7 @@ test-cov: ## Create coverage report .PHONY: test-pyspark test-pyspark: - uv run --no-sync pytest -m 'pyspark and integration' + uv run --no-sync pytest -m 'pyarrow and pyspark and integration' .PHONY: build-documentation build-documentation: ## Build documentation with Sphinx diff --git a/python/deltalake/__init__.py b/python/deltalake/__init__.py index adbfb1d3c8..adb462a888 100644 --- a/python/deltalake/__init__.py +++ b/python/deltalake/__init__.py @@ -1,7 +1,12 @@ -from deltalake._internal import TableFeatures, __version__, rust_core_version +from deltalake._internal import ( + TableFeatures, + Transaction, + __version__, + rust_core_version, +) from deltalake.query import QueryBuilder from deltalake.schema import DataType, Field, Schema -from deltalake.table import DeltaTable, Metadata, Transaction +from deltalake.table import DeltaTable, Metadata from deltalake.transaction import CommitProperties, PostCommitHookProperties from deltalake.writer import ( BloomFilterProperties, diff --git a/python/deltalake/_internal.pyi b/python/deltalake/_internal.pyi index dc2a70d99d..dc149642f3 100644 --- a/python/deltalake/_internal.pyi +++ b/python/deltalake/_internal.pyi @@ -9,10 +9,15 @@ from typing import ( Union, ) -import pyarrow -import pyarrow.fs as fs +from arro3.core import DataType as ArrowDataType +from arro3.core import Field as ArrowField +from arro3.core import RecordBatch, RecordBatchReader +from arro3.core import Schema as ArrowSchema +from arro3.core.types import ArrowSchemaExportable if TYPE_CHECKING: + import pyarrow.fs as fs + from deltalake.transaction import ( AddAction, CommitProperties, @@ -21,7 +26,6 @@ if TYPE_CHECKING: from deltalake.writer.properties import ( WriterProperties, ) - __version__: str class TableFeatures(Enum): @@ -88,9 +92,6 @@ class RawDeltaTable: def has_files(self) -> bool: ... def get_add_file_sizes(self) -> dict[str, int]: ... def get_latest_version(self) -> int: ... - def get_earliest_version(self) -> int: ... - def get_num_index_cols(self) -> int: ... - def get_stats_columns(self) -> list[str] | None: ... def metadata(self) -> RawDeltaTableMetaData: ... def protocol_versions(self) -> list[Any]: ... def load_version(self, version: int) -> None: ... @@ -104,6 +105,7 @@ class RawDeltaTable: enforce_retention_duration: bool, commit_properties: CommitProperties | None, post_commithook_properties: PostCommitHookProperties | None, + full: bool, ) -> list[str]: ... def compact_optimize( self, @@ -159,6 +161,16 @@ class RawDeltaTable: raise_if_not_exists: bool, commit_properties: CommitProperties | None, ) -> None: ... + def set_table_name( + self, + name: str, + commit_properties: CommitProperties | None = None, + ) -> None: ... + def set_table_description( + self, + description: str, + commit_properties: CommitProperties | None = None, + ) -> None: ... def restore( self, target: Any | None, @@ -169,10 +181,12 @@ class RawDeltaTable: def history(self, limit: int | None) -> list[str]: ... def update_incremental(self) -> None: ... def dataset_partitions( - self, schema: pyarrow.Schema, partition_filters: FilterConjunctionType | None + self, + schema: ArrowSchemaExportable, + partition_filters: FilterConjunctionType | None, ) -> list[Any]: ... def create_checkpoint(self) -> None: ... - def get_add_actions(self, flatten: bool) -> pyarrow.RecordBatch: ... + def get_add_actions(self, flatten: bool) -> RecordBatch: ... def delete( self, predicate: str | None, @@ -197,7 +211,8 @@ class RawDeltaTable: ) -> str: ... def create_merge_builder( self, - source: pyarrow.RecordBatchReader, + source: RecordBatchReader, + batch_schema: ArrowSchema, predicate: str, source_alias: str | None, target_alias: str | None, @@ -217,13 +232,12 @@ class RawDeltaTable: add_actions: list[AddAction], mode: str, partition_by: list[str], - schema: pyarrow.Schema, + schema: Schema, partitions_filters: FilterType | None, commit_properties: CommitProperties | None, post_commithook_properties: PostCommitHookProperties | None, ) -> None: ... def cleanup_metadata(self) -> None: ... - def check_can_write_timestamp_ntz(self, schema: pyarrow.Schema) -> None: ... def load_cdf( self, columns: list[str] | None = None, @@ -233,8 +247,8 @@ class RawDeltaTable: starting_timestamp: str | None = None, ending_timestamp: str | None = None, allow_out_of_range: bool = False, - ) -> pyarrow.RecordBatchReader: ... - def transaction_versions(self) -> dict[str, Transaction]: ... + ) -> RecordBatchReader: ... + def transaction_version(self, app_id: str) -> int | None: ... def set_column_metadata( self, column: str, @@ -245,7 +259,8 @@ class RawDeltaTable: def __datafusion_table_provider__(self) -> Any: ... def write( self, - data: pyarrow.RecordBatchReader, + data: RecordBatchReader, + batch_schema: ArrowSchema, partition_by: list[str] | None, mode: str, schema_mode: str | None, @@ -262,7 +277,7 @@ class RawDeltaTable: def rust_core_version() -> str: ... def create_table_with_add_actions( table_uri: str, - schema: pyarrow.Schema, + schema: Schema, add_actions: list[AddAction], mode: Literal["error", "append", "overwrite", "ignore"], partition_by: list[str], @@ -275,7 +290,8 @@ def create_table_with_add_actions( ) -> None: ... def write_to_deltalake( table_uri: str, - data: pyarrow.RecordBatchReader, + data: RecordBatchReader, + batch_schema: ArrowSchema, partition_by: list[str] | None, mode: str, schema_mode: str | None, @@ -291,7 +307,7 @@ def write_to_deltalake( ) -> None: ... def convert_to_deltalake( uri: str, - partition_schema: pyarrow.Schema | None, + partition_schema: Schema | None, partition_strategy: Literal["hive"] | None, name: str | None, description: str | None, @@ -302,7 +318,7 @@ def convert_to_deltalake( ) -> None: ... def create_deltalake( table_uri: str, - schema: pyarrow.Schema, + schema: Schema, partition_by: list[str], mode: str, raise_if_key_not_exists: bool, @@ -313,7 +329,6 @@ def create_deltalake( commit_properties: CommitProperties | None, post_commithook_properties: PostCommitHookProperties | None, ) -> None: ... -def batch_distinct(batch: pyarrow.RecordBatch) -> pyarrow.RecordBatch: ... def get_num_idx_cols_and_stats_columns( table: RawDeltaTable | None, configuration: Mapping[str, str | None] | None ) -> tuple[int, list[str] | None]: ... @@ -322,7 +337,7 @@ class PyMergeBuilder: source_alias: str target_alias: str merge_schema: bool - arrow_schema: pyarrow.Schema + arrow_schema: ArrowSchema def when_matched_update( self, updates: dict[str, str], predicate: str | None @@ -365,7 +380,7 @@ class PrimitiveType: * "binary", * "date", * "timestamp", - * "timestampNtz", + * "timestamp_ntz", * "decimal(, )" Max: decimal(38,38) Args: @@ -390,20 +405,31 @@ class PrimitiveType: Returns: a PrimitiveType type """ - def to_pyarrow(self) -> pyarrow.DataType: - """Get the equivalent PyArrow type (pyarrow.DataType)""" + def to_arrow(self) -> ArrowDataType: + """Get the equivalent arro3 DataType (arro3.core.DataType)""" + @staticmethod - def from_pyarrow(type: pyarrow.DataType) -> PrimitiveType: - """Create a PrimitiveType from a PyArrow datatype + def from_arrow(type: ArrowSchemaExportable) -> PrimitiveType: + """Create a PrimitiveType from an `ArrowSchemaExportable` datatype - Will raise `TypeError` if the PyArrow type is not a primitive type. + Will raise `TypeError` if the arrow type is not a primitive type. Args: - type: A PyArrow DataType + type: an object that is `ArrowSchemaExportable` Returns: a PrimitiveType """ + def __arrow_c_schema__(self) -> object: + """ + An implementation of the [Arrow PyCapsule + Interface](https://arrow.apache.org/docs/format/CDataInterface/PyCapsuleInterface.html). + This dunder method should not be called directly, but enables zero-copy data + transfer to other Python libraries that understand Arrow memory. + + For example, you can call [`pyarrow.schema()`][pyarrow.schema] to convert this + array into a pyarrow schema, without copying memory. + """ class ArrayType: """An Array (List) DataType @@ -467,22 +493,32 @@ class ArrayType: # Returns ArrayType(PrimitiveType("integer"), contains_null=False) ``` """ - def to_pyarrow( + def to_arrow( self, - ) -> pyarrow.ListType: - """Get the equivalent PyArrow type.""" + ) -> ArrowDataType: + """Get the equivalent arro3 type.""" @staticmethod - def from_pyarrow(type: pyarrow.ListType) -> ArrayType: - """Create an ArrayType from a pyarrow.ListType. + def from_arrow(type: ArrowSchemaExportable) -> ArrayType: + """Create an ArrayType from an `ArrowSchemaExportable` datatype. - Will raise `TypeError` if a different PyArrow DataType is provided. + Will raise `TypeError` if a different arrow DataType is provided. Args: - type: The PyArrow ListType + type: an object that is `ArrowSchemaExportable` Returns: an ArrayType """ + def __arrow_c_schema__(self) -> object: + """ + An implementation of the [Arrow PyCapsule + Interface](https://arrow.apache.org/docs/format/CDataInterface/PyCapsuleInterface.html). + This dunder method should not be called directly, but enables zero-copy data + transfer to other Python libraries that understand Arrow memory. + + For example, you can call [`pyarrow.schema()`][pyarrow.schema] to convert this + array into a pyarrow schema, without copying memory. + """ class MapType: """A map data type @@ -565,20 +601,31 @@ class MapType: # Returns MapType(PrimitiveType("integer"), PrimitiveType("string"), value_contains_null=True) ``` """ - def to_pyarrow(self) -> pyarrow.MapType: - """Get the equivalent PyArrow data type.""" + def to_arrow(self) -> ArrowDataType: + """Get the equivalent arro3 data type.""" + @staticmethod - def from_pyarrow(type: pyarrow.MapType) -> MapType: - """Create a MapType from a PyArrow MapType. + def from_arrow(type: ArrowSchemaExportable) -> MapType: + """Create a MapType from an `ArrowSchemaExportable` datatype Will raise `TypeError` if passed a different type. Args: - type: the PyArrow MapType + type: an object that is `ArrowSchemaExportable` Returns: a MapType """ + def __arrow_c_schema__(self) -> object: + """ + An implementation of the [Arrow PyCapsule + Interface](https://arrow.apache.org/docs/format/CDataInterface/PyCapsuleInterface.html). + This dunder method should not be called directly, but enables zero-copy data + transfer to other Python libraries that understand Arrow memory. + + For example, you can call [`pyarrow.schema()`][pyarrow.schema] to convert this + array into a pyarrow schema, without copying memory. + """ class Field: """A field in a Delta StructType or Schema @@ -663,24 +710,35 @@ class Field: # Returns Field(col, PrimitiveType("integer"), nullable=True) ``` """ - def to_pyarrow(self) -> pyarrow.Field: - """Convert to an equivalent PyArrow field + def to_arrow(self) -> ArrowField: + """Convert to an equivalent arro3 field Note: This currently doesn't preserve field metadata. Returns: - a pyarrow Field + a arro3 Field """ @staticmethod - def from_pyarrow(field: pyarrow.Field) -> Field: - """Create a Field from a PyArrow field + def from_arrow(field: ArrowSchemaExportable) -> Field: + """Create a Field from an object with an `ArrowSchemaExportable` field + Note: This currently doesn't preserve field metadata. Args: - field: a PyArrow Field + field: a Field object that is `ArrowSchemaExportable` Returns: a Field """ + def __arrow_c_schema__(self) -> object: + """ + An implementation of the [Arrow PyCapsule + Interface](https://arrow.apache.org/docs/format/CDataInterface/PyCapsuleInterface.html). + This dunder method should not be called directly, but enables zero-copy data + transfer to other Python libraries that understand Arrow memory. + + For example, you can call [`pyarrow.schema()`][pyarrow.schema] to convert this + array into a pyarrow schema, without copying memory. + """ class StructType: """A struct datatype, containing one or more subfields @@ -732,24 +790,31 @@ class StructType: # Returns StructType([Field(x, PrimitiveType("integer"), nullable=True)]) ``` """ - def to_pyarrow(self) -> pyarrow.StructType: - """Get the equivalent PyArrow StructType + def to_arrow(self) -> ArrowDataType: + """Get the equivalent arro3 DataType (arro3.core.DataType)""" - Returns: - a PyArrow StructType - """ @staticmethod - def from_pyarrow(type: pyarrow.StructType) -> StructType: - """Create a new StructType from a PyArrow struct type. + def from_arrow(type: ArrowSchemaExportable) -> StructType: + """Create a new StructType from an `ArrowSchemaExportable` datatype Will raise `TypeError` if a different data type is provided. Args: - type: a PyArrow struct type. + type: a struct type object that is `ArrowSchemaExportable` Returns: a StructType """ + def __arrow_c_schema__(self) -> object: + """ + An implementation of the [Arrow PyCapsule + Interface](https://arrow.apache.org/docs/format/CDataInterface/PyCapsuleInterface.html). + This dunder method should not be called directly, but enables zero-copy data + transfer to other Python libraries that understand Arrow memory. + + For example, you can call [`pyarrow.schema()`][pyarrow.schema] to convert this + array into a pyarrow schema, without copying memory. + """ class Schema: def __init__(self, fields: list[Field]) -> None: ... @@ -790,29 +855,41 @@ class Schema: # Returns Schema([Field(x, PrimitiveType("integer"), nullable=True)]) ``` """ - def to_pyarrow(self, as_large_types: bool = False) -> pyarrow.Schema: - """Return equivalent PyArrow schema + def to_arrow(self, as_large_types: bool = False) -> ArrowSchema: + """Return equivalent arro3 schema Args: as_large_types: get schema with all variable size types (list, binary, string) as large variants (with int64 indices). This is for compatibility with systems like Polars that only support the large versions of Arrow types. Returns: - a PyArrow Schema + an arro3 Schema """ + @staticmethod - def from_pyarrow(type: pyarrow.Schema) -> Schema: - """Create a [Schema][deltalake.schema.Schema] from a PyArrow Schema type + def from_arrow(type: ArrowSchemaExportable) -> Schema: + """Create a [Schema][deltalake.schema.Schema] from a schema that implements Arrow C Data Interface. - Will raise `TypeError` if the PyArrow type is not a primitive type. + Will raise `TypeError` if one of the Arrow type is not a primitive type. Args: - type: A PyArrow Schema + type: an object that is `ArrowSchemaExportable` Returns: a Schema """ + def __arrow_c_schema__(self) -> object: + """ + An implementation of the [Arrow PyCapsule + Interface](https://arrow.apache.org/docs/format/CDataInterface/PyCapsuleInterface.html). + This dunder method should not be called directly, but enables zero-copy data + transfer to other Python libraries that understand Arrow memory. + + For example, you can call [`pyarrow.schema()`][pyarrow.schema] to convert this + array into a pyarrow schema, without copying memory. + """ + class ObjectInputFile: @property def closed(self) -> bool: ... @@ -915,11 +992,7 @@ class DeltaFileSystemHandler: class PyQueryBuilder: def __init__(self) -> None: ... def register(self, table_name: str, delta_table: RawDeltaTable) -> None: ... - def execute(self, sql: str) -> list[pyarrow.RecordBatch]: ... - -class DeltaDataChecker: - def __init__(self, invariants: list[tuple[str, str]]) -> None: ... - def check_batch(self, batch: pyarrow.RecordBatch) -> None: ... + def execute(self, sql: str) -> RecordBatchReader: ... class DeltaError(Exception): """The base class for Delta-specific errors.""" @@ -937,7 +1010,7 @@ class CommitFailedError(DeltaError): pass class DeltaProtocolError(DeltaError): - """Raised when a violation with the Delta protocol specs ocurred.""" + """Raised when a violation with the Delta protocol specs occurred.""" pass diff --git a/python/deltalake/fs/__init__.py b/python/deltalake/fs/__init__.py new file mode 100644 index 0000000000..d2ef7f503f --- /dev/null +++ b/python/deltalake/fs/__init__.py @@ -0,0 +1,3 @@ +from deltalake.fs.fs_handler import DeltaStorageHandler + +__all__ = ["DeltaStorageHandler"] diff --git a/python/deltalake/fs.py b/python/deltalake/fs/_base_handler.py similarity index 80% rename from python/deltalake/fs.py rename to python/deltalake/fs/_base_handler.py index 446f5bd0f1..d887c0a681 100644 --- a/python/deltalake/fs.py +++ b/python/deltalake/fs/_base_handler.py @@ -1,18 +1,19 @@ from __future__ import annotations +from abc import abstractmethod from collections.abc import Mapping -from typing import Any - -import pyarrow as pa -from pyarrow.fs import FileInfo, FileSelector, FileSystemHandler +from typing import TYPE_CHECKING, Any from deltalake._internal import DeltaFileSystemHandler, RawDeltaTable +if TYPE_CHECKING: + import pyarrow as pa + from pyarrow.fs import FileInfo, FileSelector + -# NOTE we need to inherit form FileSystemHandler to pass pyarrow's internal type checks. -class DeltaStorageHandler(FileSystemHandler): +class BaseDeltaStorageHandler: """ - DeltaStorageHandler is a concrete implementations of a PyArrow FileSystemHandler. + BaseDeltaStorageHandler is a concrete implementations of a PyArrow FileSystemHandler. """ def __init__( @@ -31,7 +32,7 @@ def from_table( table: RawDeltaTable, options: dict[str, str] | None = None, known_sizes: dict[str, int] | None = None, - ) -> DeltaStorageHandler: + ) -> BaseDeltaStorageHandler: self = cls.__new__(cls) self._handler = DeltaFileSystemHandler.from_table(table, options, known_sizes) return self @@ -65,21 +66,27 @@ def equals(self, other: Any) -> bool: return self._handler.equals(other) def delete_dir_contents( - self, path: str, *, accept_root_dir: bool = False, missing_dir_ok: bool = False + self, + path: str, + *, + accept_root_dir: bool = False, + missing_dir_ok: bool = False, ) -> None: """Delete a directory's contents, recursively. Like delete_dir, but doesn't delete the directory itself. """ return self._handler.delete_dir_contents( - path=path, accept_root_dir=accept_root_dir, missing_dir_ok=missing_dir_ok + path=path, + accept_root_dir=accept_root_dir, + missing_dir_ok=missing_dir_ok, ) def delete_root_dir_contents(self) -> None: """Delete the root directory contents, recursively.""" return self._handler.delete_root_dir_contents() - def get_file_info(self, paths: list[str]) -> list[FileInfo]: + def get_file_info(self, paths: list[str]) -> list["FileInfo"]: """Get info for the given files. A non-existing or unreachable file returns a FileStat object and has a FileType of value NotFound. @@ -100,7 +107,8 @@ def normalize_path(self, path: str) -> str: """Normalize filesystem path.""" return self._handler.normalize_path(path) - def open_input_file(self, path: str) -> pa.PythonFile: + @abstractmethod + def open_input_file(self, path: str) -> "pa.PythonFile": """ Open an input file for random access reading. @@ -110,9 +118,9 @@ def open_input_file(self, path: str) -> pa.PythonFile: Returns: NativeFile """ - return pa.PythonFile(self._handler.open_input_file(path)) - def open_input_stream(self, path: str) -> pa.PythonFile: + @abstractmethod + def open_input_stream(self, path: str) -> "pa.PythonFile": """ Open an input stream for sequential reading. @@ -122,11 +130,11 @@ def open_input_stream(self, path: str) -> pa.PythonFile: Returns: NativeFile """ - return pa.PythonFile(self._handler.open_input_file(path)) + @abstractmethod def open_output_stream( self, path: str, metadata: dict[str, str] | None = None - ) -> pa.PythonFile: + ) -> "pa.PythonFile": """ Open an output stream for sequential writing. @@ -139,9 +147,8 @@ def open_output_stream( Returns: NativeFile """ - return pa.PythonFile(self._handler.open_output_stream(path, metadata)) - def get_file_info_selector(self, selector: FileSelector) -> list[FileInfo]: + def get_file_info_selector(self, selector: "FileSelector") -> list["FileInfo"]: """ Get info for the files defined by FileSelector. diff --git a/python/deltalake/fs/fs_handler.py b/python/deltalake/fs/fs_handler.py new file mode 100644 index 0000000000..fad98c413b --- /dev/null +++ b/python/deltalake/fs/fs_handler.py @@ -0,0 +1,97 @@ +from __future__ import annotations + +from collections.abc import Mapping + +from deltalake._internal import RawDeltaTable +from deltalake.fs._base_handler import BaseDeltaStorageHandler + +try: + import pyarrow as pa + from pyarrow.fs import FileSystemHandler + + PYARROW_AVAILABLE = True +except ImportError as e: + if "pyarrow" in str(e): + PYARROW_AVAILABLE = False + else: + raise + + +# NOTE we need to inherit form FileSystemHandler to pass pyarrow's internal type checks. +if PYARROW_AVAILABLE: + + class DeltaStorageHandler(BaseDeltaStorageHandler, FileSystemHandler): + """ + DeltaStorageHandler is a concrete implementations of a PyArrow FileSystemHandler. + """ + + def open_input_file(self, path: str) -> pa.PythonFile: + """ + Open an input file for random access reading. + + Args: + path: The source to open for reading. + + Returns: + NativeFile + """ + return pa.PythonFile(self._handler.open_input_file(path)) + + def open_input_stream(self, path: str) -> pa.PythonFile: + """ + Open an input stream for sequential reading. + + Args: + path: The source to open for reading. + + Returns: + NativeFile + """ + return pa.PythonFile(self._handler.open_input_file(path)) + + def open_output_stream( + self, path: str, metadata: dict[str, str] | None = None + ) -> pa.PythonFile: + """ + Open an output stream for sequential writing. + + If the target already exists, existing data is truncated. + + Args: + path: The source to open for writing. + metadata: If not None, a mapping of string keys to string values. + + Returns: + NativeFile + """ + return pa.PythonFile(self._handler.open_output_stream(path, metadata)) + + def open_append_stream(self, path: str, metadata: Mapping[str, str]) -> None: + raise NotImplementedError +else: + + class DeltaStorageHandler(BaseDeltaStorageHandler): # type: ignore[no-redef] + """ + DeltaStorageHandler is a concrete implementations of a PyArrow FileSystemHandler. + """ + + def __init__( + self, + table_uri: str, + options: dict[str, str] | None = None, + known_sizes: dict[str, int] | None = None, + ) -> None: + raise ImportError( + "DeltaStorageHandler requires pyarrow. Please install deltalake[pyarrow] to use this class." + ) + + @classmethod + def from_table( + cls, + table: RawDeltaTable, + options: dict[str, str] | None = None, + known_sizes: dict[str, int] | None = None, + ) -> "DeltaStorageHandler": + raise ImportError( + "DeltaStorageHandler requires pyarrow. Please install deltalake[pyarrow] to use this class." + ) diff --git a/python/deltalake/query.py b/python/deltalake/query.py index 5660a3bc19..3f1f3e3445 100644 --- a/python/deltalake/query.py +++ b/python/deltalake/query.py @@ -1,31 +1,23 @@ from __future__ import annotations import logging -import warnings -import pyarrow +from arro3.core import RecordBatchReader from deltalake._internal import PyQueryBuilder from deltalake.table import DeltaTable -from deltalake.warnings import ExperimentalWarning logger = logging.getLogger(__name__) class QueryBuilder: """ - QueryBuilder is an experimental API which exposes Apache DataFusion SQL to Python users of the deltalake library. - - This API is subject to change. + QueryBuilder is an API which exposes Apache DataFusion SQL to Python users of the deltalake library. >>> qb = QueryBuilder() """ def __init__(self) -> None: - warnings.warn( - "QueryBuilder is experimental and subject to change", - category=ExperimentalWarning, - ) self._query_builder = PyQueryBuilder() def register(self, table_name: str, delta_table: DeltaTable) -> QueryBuilder: @@ -36,12 +28,11 @@ def register(self, table_name: str, delta_table: DeltaTable) -> QueryBuilder: For example: - >>> tmp = getfixture('tmp_path') - >>> import pyarrow as pa - >>> from deltalake import DeltaTable, QueryBuilder - >>> dt = DeltaTable.create(table_uri=tmp, schema=pa.schema([pa.field('name', pa.string())])) - >>> qb = QueryBuilder().register('test', dt) - >>> assert qb is not None + ```python + from deltalake import DeltaTable, QueryBuilder + dt = DeltaTable("my_table") + qb = QueryBuilder().register('test', dt) + ``` """ self._query_builder.register( table_name=table_name, @@ -49,73 +40,15 @@ def register(self, table_name: str, delta_table: DeltaTable) -> QueryBuilder: ) return self - def execute(self, sql: str) -> QueryResult: + def execute(self, sql: str) -> RecordBatchReader: """ Prepares the sql query to be executed. For example: - - >>> tmp = getfixture('tmp_path') - >>> import pyarrow as pa - >>> from deltalake import DeltaTable, QueryBuilder - >>> dt = DeltaTable.create(table_uri=tmp, schema=pa.schema([pa.field('name', pa.string())])) - >>> qb = QueryBuilder().register('test', dt) - >>> results = qb.execute('SELECT * FROM test') - >>> assert isinstance(results, QueryResult) - """ - return QueryResult(self._query_builder, sql) - - def sql(self, sql: str) -> QueryResult: - """ - Convenience method for `execute()` method. - - For example: - - >>> tmp = getfixture('tmp_path') - >>> import pyarrow as pa - >>> from deltalake import DeltaTable, QueryBuilder - >>> dt = DeltaTable.create(table_uri=tmp, schema=pa.schema([pa.field('name', pa.string())])) - >>> qb = QueryBuilder().register('test', dt) - >>> query = 'SELECT * FROM test' - >>> assert qb.execute(query).fetchall() == qb.sql(query).fetchall() - """ - return self.execute(sql) - - -class QueryResult: - def __init__(self, query_builder: PyQueryBuilder, sql: str) -> None: - self._query_builder = query_builder - self._sql_query = sql - - def show(self) -> None: - """ - Execute the query and prints the output in the console. - - For example: - - >>> tmp = getfixture('tmp_path') - >>> import pyarrow as pa - >>> from deltalake import DeltaTable, QueryBuilder - >>> dt = DeltaTable.create(table_uri=tmp, schema=pa.schema([pa.field('name', pa.string())])) - >>> qb = QueryBuilder().register('test', dt) - >>> results = qb.execute('SELECT * FROM test').show() - """ - records = self.fetchall() - if len(records) > 0: - print(pyarrow.Table.from_batches(records)) - else: - logger.info("The executed query contains no records.") - - def fetchall(self) -> list[pyarrow.RecordBatch]: - """ - Execute the query and return a list of record batches. - - >>> tmp = getfixture('tmp_path') - >>> import pyarrow as pa - >>> from deltalake import DeltaTable, QueryBuilder - >>> dt = DeltaTable.create(table_uri=tmp, schema=pa.schema([pa.field('name', pa.string())])) - >>> qb = QueryBuilder().register('test', dt) - >>> results = qb.execute('SELECT * FROM test').fetchall() - >>> assert results is not None + ```python + from deltalake import DeltaTable, QueryBuilder + dt = DeltaTable("my_table") + data = QueryBuilder().register('test', dt).execute("select * from my_table").read_all() + ``` """ - return self._query_builder.execute(self._sql_query) + return self._query_builder.execute(sql) diff --git a/python/deltalake/table.py b/python/deltalake/table.py index 4413a381eb..973a472b3c 100644 --- a/python/deltalake/table.py +++ b/python/deltalake/table.py @@ -15,44 +15,38 @@ Union, ) -import pyarrow -import pyarrow.dataset as ds -import pyarrow.fs as pa_fs -from pyarrow.dataset import ( - Expression, - FileSystemDataset, - ParquetFileFormat, - ParquetFragmentScanOptions, - ParquetReadOptions, +from arro3.core import RecordBatch, RecordBatchReader +from arro3.core.types import ( + ArrowArrayExportable, + ArrowSchemaExportable, + ArrowStreamExportable, ) +from deprecated import deprecated from deltalake._internal import ( DeltaError, PyMergeBuilder, RawDeltaTable, TableFeatures, - Transaction, ) from deltalake._internal import create_deltalake as _create_deltalake from deltalake._util import encode_partition_value from deltalake.exceptions import DeltaProtocolError -from deltalake.fs import DeltaStorageHandler from deltalake.schema import Field as DeltaField from deltalake.schema import Schema as DeltaSchema -from deltalake.writer._conversion import ( - ArrowSchemaConversionMode, - ArrowStreamExportable, - _convert_data_and_schema, -) - -try: - from pyarrow.parquet import filters_to_expression # pyarrow >= 10.0.0 -except ImportError: - from pyarrow.parquet import _filters_to_expression as filters_to_expression +from deltalake.writer._conversion import _convert_arro3_schema_to_delta if TYPE_CHECKING: import os + import pandas as pd + import pyarrow + import pyarrow.fs as pa_fs + from pyarrow.dataset import ( + Expression, + ParquetReadOptions, + ) + from deltalake.transaction import ( AddAction, CommitProperties, @@ -61,13 +55,6 @@ from deltalake.writer.properties import WriterProperties -try: - import pandas as pd -except ModuleNotFoundError: - _has_pandas = False -else: - _has_pandas = True - MAX_SUPPORTED_PYARROW_WRITER_VERSION = 7 NOT_SUPPORTED_PYARROW_WRITER_VERSIONS = [3, 4, 5, 6] SUPPORTED_WRITER_FEATURES = {"appendOnly", "invariants", "timestampNtz"} @@ -198,7 +185,7 @@ def is_deltatable( def create( cls, table_uri: str | Path, - schema: pyarrow.Schema | DeltaSchema, + schema: DeltaSchema | ArrowSchemaExportable, mode: Literal["error", "append", "overwrite", "ignore"] = "error", partition_by: list[str] | str | None = None, name: str | None = None, @@ -246,14 +233,15 @@ def create( ) ``` """ - if isinstance(schema, DeltaSchema): - schema = schema.to_pyarrow() if isinstance(partition_by, str): partition_by = [partition_by] if isinstance(table_uri, Path): table_uri = str(table_uri) + if not isinstance(schema, DeltaSchema): + schema = DeltaSchema.from_arrow(schema) + _create_deltalake( table_uri, schema, @@ -297,6 +285,10 @@ def partitions( partitions.append({k: v for (k, v) in partition}) return partitions + @deprecated( + version="1.0.0", + reason="Not compatible with modern delta features (e.g. shallow clones). Use `file_uris` instead.", + ) def files( self, partition_filters: list[tuple[str, str, Any]] | None = None ) -> list[str]: @@ -424,7 +416,25 @@ def load_cdf( columns: list[str] | None = None, predicate: str | None = None, allow_out_of_range: bool = False, - ) -> pyarrow.RecordBatchReader: + ) -> RecordBatchReader: + """ + Load the Change Data Feed (CDF) from the Delta table as a stream of record batches. + + Parameters: + starting_version (int): The version of the Delta table to start reading CDF from. + ending_version (int | None): The version to stop reading CDF at. If None, reads up to the latest version. + starting_timestamp (str | None): An ISO 8601 timestamp to start reading CDF from. Ignored if starting_version is provided. + ending_timestamp (str | None): An ISO 8601 timestamp to stop reading CDF at. Ignored if ending_version is provided. + columns (list[str] | None): A list of column names to include in the output. If None, all columns are included. + predicate (str | None): An optional SQL predicate to filter the output rows. + allow_out_of_range (bool): If True, does not raise an error when specified versions or timestamps are outside the table's history. + + Returns: + RecordBatchReader: An Arrow RecordBatchReader that streams the resulting change data. + + Raises: + ValueError: If input parameters are invalid or if the specified range is not found (unless allow_out_of_range is True). + """ return self._table.load_cdf( columns=columns, predicate=predicate, @@ -516,6 +526,7 @@ def vacuum( enforce_retention_duration: bool = True, post_commithook_properties: PostCommitHookProperties | None = None, commit_properties: CommitProperties | None = None, + full: bool = False, ) -> list[str]: """ Run the Vacuum command on the Delta Table: list and delete files no longer referenced by the Delta table and are older than the retention threshold. @@ -526,6 +537,7 @@ def vacuum( enforce_retention_duration: when disabled, accepts retention hours smaller than the value from `delta.deletedFileRetentionDuration`. post_commithook_properties: properties for the post commit hook. If None, default values are used. commit_properties: properties of the transaction commit. If None, default values are used. + full: when set to True, will perform a "full" vacuum and remove all files not referenced in the transaction log Returns: the list of files no longer referenced by the Delta Table and are older than the retention threshold. """ @@ -539,6 +551,7 @@ def vacuum( enforce_retention_duration, commit_properties, post_commithook_properties, + full, ) def update( @@ -669,12 +682,7 @@ def alter( def merge( self, - source: pyarrow.Table - | pyarrow.RecordBatch - | pyarrow.RecordBatchReader - | ds.Dataset - | ArrowStreamExportable - | pd.DataFrame, + source: ArrowStreamExportable | ArrowArrayExportable, predicate: str, source_alias: str | None = None, target_alias: str | None = None, @@ -705,15 +713,13 @@ def merge( Returns: TableMerger: TableMerger Object """ - data, schema = _convert_data_and_schema( - data=source, - schema=None, - conversion_mode=ArrowSchemaConversionMode.PASSTHROUGH, - ) - data = pyarrow.RecordBatchReader.from_batches(schema, (batch for batch in data)) + + source = RecordBatchReader.from_arrow(source) + compatible_delta_schema = _convert_arro3_schema_to_delta(source.schema) py_merge_builder = self._table.create_merge_builder( source=source, + batch_schema=compatible_delta_schema, predicate=predicate, source_alias=source_alias, target_alias=target_alias, @@ -769,7 +775,7 @@ def to_pyarrow_dataset( parquet_read_options: ParquetReadOptions | None = None, schema: pyarrow.Schema | None = None, as_large_types: bool = False, - ) -> pyarrow.dataset.Dataset: + ) -> "pyarrow.dataset.Dataset": """ Build a PyArrow Dataset using data from the DeltaTable. @@ -805,6 +811,16 @@ def to_pyarrow_dataset( Returns: the PyArrow dataset in PyArrow """ + try: + from pyarrow.dataset import ( + FileSystemDataset, + ParquetFileFormat, + ParquetFragmentScanOptions, + ) + except ImportError: + raise ImportError( + "Pyarrow is required, install deltalake[pyarrow] for pyarrow read functionality." + ) if not self._table.has_files(): raise DeltaError("Table is instantiated without files.") @@ -815,7 +831,8 @@ def to_pyarrow_dataset( ): raise DeltaProtocolError( f"The table's minimum reader version is {table_protocol.min_reader_version} " - f"but deltalake only supports version 1 or {MAX_SUPPORTED_READER_VERSION} with these reader features: {SUPPORTED_READER_FEATURES}" + f"but deltalake only supports version 1 or {MAX_SUPPORTED_READER_VERSION} " + f"with these reader features: {SUPPORTED_READER_FEATURES}" ) if ( table_protocol.min_reader_version >= 3 @@ -826,8 +843,15 @@ def to_pyarrow_dataset( ) if len(missing_features) > 0: raise DeltaProtocolError( - f"The table has set these reader features: {missing_features} but these are not yet supported by the deltalake reader." + f"The table has set these reader features: {missing_features} " + "but these are not yet supported by the deltalake reader." ) + + import pyarrow + import pyarrow.fs as pa_fs + + from deltalake.fs import DeltaStorageHandler + if not filesystem: filesystem = pa_fs.PyFileSystem( DeltaStorageHandler.from_table( @@ -836,12 +860,16 @@ def to_pyarrow_dataset( self._table.get_add_file_sizes(), ) ) + format = ParquetFileFormat( read_options=parquet_read_options, default_fragment_scan_options=ParquetFragmentScanOptions(pre_buffer=True), ) - schema = schema or self.schema().to_pyarrow(as_large_types=as_large_types) + if schema is None: + schema = pyarrow.schema( + self.schema().to_arrow(as_large_types=as_large_types) + ) fragments = [ format.make_fragment( @@ -871,7 +899,7 @@ def to_pyarrow_table( columns: list[str] | None = None, filesystem: str | pa_fs.FileSystem | None = None, filters: FilterType | Expression | None = None, - ) -> pyarrow.Table: + ) -> "pyarrow.Table": """ Build a PyArrow Table using data from the DeltaTable. @@ -881,6 +909,13 @@ def to_pyarrow_table( filesystem: A concrete implementation of the Pyarrow FileSystem or a fsspec-compatible interface. If None, the first file path will be used to determine the right FileSystem filters: A disjunctive normal form (DNF) predicate for filtering rows, or directly a pyarrow.dataset.Expression. If you pass a filter you do not need to pass ``partitions`` """ + try: + from pyarrow.parquet import filters_to_expression # pyarrow >= 10.0.0 + except ImportError: + raise ImportError( + "Pyarrow is required, install deltalake[pyarrow] for pyarrow read functionality." + ) + if filters is not None: filters = filters_to_expression(filters) return self.to_pyarrow_dataset( @@ -894,7 +929,7 @@ def to_pandas( filesystem: str | pa_fs.FileSystem | None = None, filters: FilterType | Expression | None = None, types_mapper: Callable[[pyarrow.DataType], Any] | None = None, - ) -> pd.DataFrame: + ) -> "pd.DataFrame": """ Build a pandas dataframe using data from the DeltaTable. @@ -943,7 +978,7 @@ def _stringify_partition_values( out.append((field, op, str_value)) return out - def get_add_actions(self, flatten: bool = False) -> pyarrow.RecordBatch: + def get_add_actions(self, flatten: bool = False) -> RecordBatch: """ Return a dataframe with all current add actions. @@ -991,7 +1026,7 @@ def delete( post_commithook_properties: PostCommitHookProperties | None = None, commit_properties: CommitProperties | None = None, ) -> dict[str, Any]: - """Delete records from a Delta Table that statisfy a predicate. + """Delete records from a Delta Table that satisfy a predicate. When a predicate is not provided then all records are deleted from the Delta Table. Otherwise a scan of the Delta table is performed to mark any files @@ -1058,14 +1093,23 @@ def repair( ) return deserialized_metrics - def transaction_versions(self) -> dict[str, Transaction]: - return self._table.transaction_versions() + def transaction_version(self, app_id: str) -> int | None: + """ + Retrieve the latest transaction versions for the given application ID. + + Args: + app_id (str): The application ID for which to retrieve the latest transaction version. + + Returns: + int | None: The latest transaction version for the given application ID if it exists, otherwise None. + """ + return self._table.transaction_version(app_id) def create_write_transaction( self, actions: list[AddAction], mode: str, - schema: pyarrow.Schema, + schema: DeltaSchema | ArrowSchemaExportable, partition_by: list[str] | str | None = None, partition_filters: FilterType | None = None, commit_properties: CommitProperties | None = None, @@ -1074,6 +1118,9 @@ def create_write_transaction( if isinstance(partition_by, str): partition_by = [partition_by] + if not isinstance(schema, DeltaSchema): + schema = DeltaSchema.from_arrow(schema) + self._table.create_write_transaction( actions, mode, @@ -1269,7 +1316,7 @@ def when_matched_update_all( updates = { f"{trgt_alias}`{col.name}`": f"{src_alias}`{col.name}`" - for col in self._builder.arrow_schema + for col in self._builder.arrow_schema # type: ignore[attr-defined] if col.name not in except_columns } @@ -1486,7 +1533,7 @@ def when_not_matched_insert_all( updates = { f"{trgt_alias}`{col.name}`": f"{src_alias}`{col.name}`" - for col in self._builder.arrow_schema + for col in self._builder.arrow_schema # type: ignore[attr-defined] if col.name not in except_columns } @@ -1590,7 +1637,7 @@ def add_feature( Args: feature: Table Feature e.g. Deletion Vectors, Change Data Feed - allow_protocol_versions_increase: Allow the protocol to be implicitily bumped to reader 3 or writer 7 + allow_protocol_versions_increase: Allow the protocol to be implicitly bumped to reader 3 or writer 7 commit_properties: properties of the transaction commit. If None, default values are used. post_commithook_properties: properties for the post commit hook. If None, default values are used. @@ -1769,6 +1816,50 @@ def set_table_properties( commit_properties, ) + def set_table_name( + self, + name: str, + commit_properties: CommitProperties | None = None, + ) -> None: + """ + Set the name of the table. + + Args: + name: the name of the table + commit_properties: properties of the transaction commit. If None, default values are used. + Note: This parameter is not yet implemented and will be ignored. + + Example: + ```python + from deltalake import DeltaTable + dt = DeltaTable("test_table") + dt.alter.set_table_name("new_table_name") + ``` + """ + self.table._table.set_table_name(name, commit_properties) + + def set_table_description( + self, + description: str, + commit_properties: CommitProperties | None = None, + ) -> None: + """ + Set the description of the table. + + Args: + description: the description of the table + commit_properties: properties of the transaction commit. If None, default values are used. + Note: This parameter is not yet implemented and will be ignored. + + Example: + ```python + from deltalake import DeltaTable + dt = DeltaTable("test_table") + dt.alter.set_table_description("new_table_description") + ``` + """ + self.table._table.set_table_description(description, commit_properties) + def set_column_metadata( self, column: str, @@ -1821,7 +1912,7 @@ def compact( partition_filters: the partition filters that will be used for getting the matched files target_size: desired file size after bin-packing files, in bytes. If not provided, will attempt to read the table configuration value ``delta.targetFileSize``. - If that value isn't set, will use default value of 256MB. + If that value isn't set, will use default value of 100MB. max_concurrent_tasks: the maximum number of concurrent tasks to use for file compaction. Defaults to number of CPUs. More concurrent tasks can make compaction faster, but will also use more memory. @@ -1888,7 +1979,7 @@ def z_order( partition_filters: the partition filters that will be used for getting the matched files target_size: desired file size after bin-packing files, in bytes. If not provided, will attempt to read the table configuration value ``delta.targetFileSize``. - If that value isn't set, will use default value of 256MB. + If that value isn't set, will use default value of 100MB. max_concurrent_tasks: the maximum number of concurrent tasks to use for file compaction. Defaults to number of CPUs. More concurrent tasks can make compaction faster, but will also use more memory. diff --git a/python/deltalake/transaction.py b/python/deltalake/transaction.py index 7bf01d0aad..78ef35943d 100644 --- a/python/deltalake/transaction.py +++ b/python/deltalake/transaction.py @@ -4,8 +4,7 @@ from dataclasses import dataclass from typing import Literal -import pyarrow as pa - +from deltalake import Schema from deltalake._internal import Transaction as Transaction from deltalake._internal import ( create_table_with_add_actions as _create_table_with_add_actions, @@ -67,7 +66,7 @@ def __init__( def create_table_with_add_actions( table_uri: str, - schema: pa.Schema, + schema: Schema, add_actions: list[AddAction], mode: Literal["error", "append", "overwrite", "ignore"] = "error", partition_by: list[str] | str | None = None, diff --git a/python/deltalake/writer/_conversion.py b/python/deltalake/writer/_conversion.py index d6be8281c5..cf819c1b24 100644 --- a/python/deltalake/writer/_conversion.py +++ b/python/deltalake/writer/_conversion.py @@ -1,274 +1,77 @@ from __future__ import annotations -from collections.abc import Generator, Iterable -from enum import Enum -from typing import Protocol +from arro3.core import DataType +from arro3.core import Schema as Arro3Schema -import pyarrow as pa -import pyarrow.dataset as ds - -try: - import pandas as pd -except ModuleNotFoundError: - _has_pandas = False -else: - _has_pandas = True - -from deltalake._internal import Schema - - -class ArrowSchemaConversionMode(Enum): - NORMAL = "NORMAL" - LARGE = "LARGE" - PASSTHROUGH = "PASSTHROUGH" - - @classmethod - def from_str(cls, value: str) -> ArrowSchemaConversionMode: - try: - return cls(value.upper()) - except ValueError: - raise ValueError( - f"{value} is not a valid ArrowSchemaConversionMode. Valid values are: {[item.value for item in ArrowSchemaConversionMode]}" - ) - - -class ArrowStreamExportable(Protocol): - """Type hint for object exporting Arrow C Stream via Arrow PyCapsule Interface. - - https://arrow.apache.org/docs/format/CDataInterface/PyCapsuleInterface.html - """ - - def __arrow_c_stream__(self, requested_schema: object | None = None) -> object: ... - - -### Inspired from Pola-rs repo - licensed with MIT License, see license in python/licenses/polars_license.txt.### -def _convert_pa_schema_to_delta( - schema: pa.schema, - schema_conversion_mode: ArrowSchemaConversionMode = ArrowSchemaConversionMode.NORMAL, -) -> pa.schema: - """Convert a PyArrow schema to a schema compatible with Delta Lake. Converts unsigned to signed equivalent, and - converts all timestamps to `us` timestamps. With the boolean flag large_dtypes you can control if the schema - should keep cast normal to large types in the schema, or from large to normal. +def _convert_arro3_schema_to_delta( + schema: Arro3Schema, +) -> Arro3Schema: + """Convert a arro3 schema to a schema compatible with Delta Lake. Converts unsigned to signed equivalent, and + converts all timestamps to `us` timestamps. Args schema: Source schema - schema_conversion_mode: large mode will cast all string/binary/list to the large version arrow types, normal mode - keeps the normal version of the types. Passthrough mode keeps string/binary/list flavored types in their original - version, whether that is view/large/normal. """ - dtype_map = { - pa.uint8(): pa.int8(), - pa.uint16(): pa.int16(), - pa.uint32(): pa.int32(), - pa.uint64(): pa.int64(), - } - if schema_conversion_mode == ArrowSchemaConversionMode.LARGE: - dtype_map = { - **dtype_map, - **{ - pa.string(): pa.large_string(), - pa.string_view(): pa.large_string(), - pa.binary(): pa.large_binary(), - pa.binary_view(): pa.large_binary(), - }, - } - elif schema_conversion_mode == ArrowSchemaConversionMode.NORMAL: - dtype_map = { - **dtype_map, - **{ - pa.large_string(): pa.string(), - pa.string_view(): pa.string(), - pa.large_binary(): pa.binary(), - pa.binary_view(): pa.binary(), - }, - } - def dtype_to_delta_dtype(dtype: pa.DataType) -> pa.DataType: + def dtype_to_delta_dtype(dtype: DataType) -> DataType: # Handle nested types - if isinstance( - dtype, - ( - pa.LargeListType, - pa.ListType, - pa.FixedSizeListType, - pa.ListViewType, - pa.LargeListViewType, - ), + if ( + DataType.is_list(dtype) + or DataType.is_large_list(dtype) + or DataType.is_fixed_size_list(dtype) + or DataType.is_list_view(dtype) + or DataType.is_large_list_view(dtype) ): return list_to_delta_dtype(dtype) - elif isinstance(dtype, pa.StructType): + elif DataType.is_struct(dtype): return struct_to_delta_dtype(dtype) - elif isinstance(dtype, pa.TimestampType): + elif DataType.is_timestamp(dtype): if dtype.tz is None: - return pa.timestamp("us") + return DataType.timestamp("us") else: - return pa.timestamp("us", "UTC") - elif type(dtype) is pa.FixedSizeBinaryType: - return pa.binary() - elif isinstance(dtype, pa.ExtensionType): - return dtype.storage_type - try: - return dtype_map[dtype] - except KeyError: + return DataType.timestamp("us", tz="UTC") + elif DataType.is_fixed_size_binary(dtype): + return DataType.binary() + elif DataType.is_unsigned_integer(dtype): + if DataType.is_uint16(dtype): + return DataType.int16() + elif DataType.is_uint32(dtype): + return DataType.int32() + elif DataType.is_uint64(dtype): + return DataType.int64() + elif DataType.is_uint8(dtype): + return DataType.int8() + else: + raise NotImplementedError + else: return dtype def list_to_delta_dtype( - dtype: pa.LargeListType - | pa.ListType - | pa.ListViewType - | pa.LargeListViewType - | pa.FixedSizeListType, - ) -> pa.LargeListType | pa.ListType: + dtype: DataType, + ) -> DataType: nested_dtype = dtype.value_type - nested_dtype_cast = dtype_to_delta_dtype(nested_dtype) - if schema_conversion_mode == ArrowSchemaConversionMode.LARGE: - return pa.large_list(nested_dtype_cast) - elif schema_conversion_mode == ArrowSchemaConversionMode.NORMAL: - return pa.list_(nested_dtype_cast) - elif schema_conversion_mode == ArrowSchemaConversionMode.PASSTHROUGH: - if isinstance(dtype, pa.LargeListType): - return pa.large_list(nested_dtype_cast) - elif isinstance(dtype, pa.ListType): - return pa.list_(nested_dtype_cast) - elif isinstance(dtype, pa.FixedSizeListType): - return pa.list_(nested_dtype_cast) - elif isinstance(dtype, pa.LargeListViewType): - return pa.large_list_view(nested_dtype_cast) - elif isinstance(dtype, pa.ListViewType): - return pa.list_view(nested_dtype_cast) - else: - raise NotImplementedError + inner_field = dtype.value_field + + assert nested_dtype is not None + assert inner_field is not None + + inner_field_casted = inner_field.with_type(dtype_to_delta_dtype(nested_dtype)) + + if DataType.is_large_list(dtype): + return DataType.large_list(inner_field_casted) + elif DataType.is_fixed_size_list(dtype): + return DataType.list(inner_field_casted, dtype.list_size) + elif DataType.is_large_list_view(dtype): + return DataType.large_list_view(inner_field_casted) + elif DataType.is_list_view(dtype): + return DataType.list_view(inner_field_casted) + elif DataType.is_list(dtype): + return DataType.list(inner_field_casted) else: raise NotImplementedError - def struct_to_delta_dtype(dtype: pa.StructType) -> pa.StructType: - fields = [dtype[i] for i in range(dtype.num_fields)] - fields_cast = [f.with_type(dtype_to_delta_dtype(f.type)) for f in fields] - return pa.struct(fields_cast) - - return pa.schema([f.with_type(dtype_to_delta_dtype(f.type)) for f in schema]) - - -def _cast_schema_to_recordbatchreader( - reader: pa.RecordBatchReader, schema: pa.schema -) -> Generator[pa.RecordBatch, None, None]: - """Creates recordbatch generator.""" - for batch in reader: - batchs = pa.Table.from_batches([batch]).cast(schema).to_batches() - if len(batchs) > 0: - yield batchs[0] - - -def convert_pyarrow_recordbatchreader( - data: pa.RecordBatchReader, schema_conversion_mode: ArrowSchemaConversionMode -) -> pa.RecordBatchReader: - """Converts a PyArrow RecordBatchReader to a PyArrow RecordBatchReader with a compatible delta schema""" - schema = _convert_pa_schema_to_delta( - data.schema, schema_conversion_mode=schema_conversion_mode - ) - - data = pa.RecordBatchReader.from_batches( - schema, - _cast_schema_to_recordbatchreader(data, schema), - ) - return data - - -def convert_pyarrow_recordbatch( - data: pa.RecordBatch, schema_conversion_mode: ArrowSchemaConversionMode -) -> pa.RecordBatchReader: - """Converts a PyArrow RecordBatch to a PyArrow RecordBatchReader with a compatible delta schema""" - schema = _convert_pa_schema_to_delta( - data.schema, schema_conversion_mode=schema_conversion_mode - ) - data = pa.Table.from_batches([data]).cast(schema).to_reader() - return data - - -def convert_pyarrow_table( - data: pa.Table, schema_conversion_mode: ArrowSchemaConversionMode -) -> pa.RecordBatchReader: - """Converts a PyArrow table to a PyArrow RecordBatchReader with a compatible delta schema""" - schema = _convert_pa_schema_to_delta( - data.schema, schema_conversion_mode=schema_conversion_mode - ) - data = data.cast(schema).to_reader() - return data - - -def convert_pyarrow_dataset( - data: ds.Dataset, schema_conversion_mode: ArrowSchemaConversionMode -) -> pa.RecordBatchReader: - """Converts a PyArrow dataset to a PyArrow RecordBatchReader with a compatible delta schema""" - data = data.scanner().to_reader() - data = convert_pyarrow_recordbatchreader( - data, schema_conversion_mode=schema_conversion_mode - ) - return data - - -def _convert_data_and_schema( - data: pd.DataFrame - | ds.Dataset - | pa.Table - | pa.RecordBatch - | Iterable[pa.RecordBatch] - | pa.RecordBatchReader - | ArrowStreamExportable, - schema: pa.Schema | Schema | None, - conversion_mode: ArrowSchemaConversionMode, -) -> tuple[pa.RecordBatchReader, pa.Schema]: - if isinstance(data, pa.RecordBatchReader): - data = convert_pyarrow_recordbatchreader(data, conversion_mode) - elif isinstance(data, pa.RecordBatch): - data = convert_pyarrow_recordbatch(data, conversion_mode) - elif isinstance(data, pa.Table): - data = convert_pyarrow_table(data, conversion_mode) - elif isinstance(data, ds.Dataset): - data = convert_pyarrow_dataset(data, conversion_mode) - elif _has_pandas and isinstance(data, pd.DataFrame): - if schema is not None: - data = convert_pyarrow_table( - pa.Table.from_pandas(data, schema=schema), conversion_mode - ) - else: - data = convert_pyarrow_table(pa.Table.from_pandas(data), conversion_mode) - elif hasattr(data, "__arrow_c_array__"): - data = convert_pyarrow_recordbatch( - pa.record_batch(data), # type:ignore[attr-defined] - conversion_mode, - ) - elif hasattr(data, "__arrow_c_stream__"): - if not hasattr(pa.RecordBatchReader, "from_stream"): - raise ValueError( - "pyarrow 15 or later required to read stream via pycapsule interface" - ) - - data = convert_pyarrow_recordbatchreader( - pa.RecordBatchReader.from_stream(data), conversion_mode - ) - elif isinstance(data, Iterable): - if schema is None: - raise ValueError("You must provide schema if data is Iterable") - else: - raise TypeError( - f"{type(data).__name__} is not a valid input. Only PyArrow RecordBatchReader, RecordBatch, Iterable[RecordBatch], Table, Dataset or Pandas DataFrame or objects implementing the Arrow PyCapsule Interface are valid inputs for source." - ) - from deltalake.schema import Schema - - if ( - isinstance(schema, Schema) - and conversion_mode == ArrowSchemaConversionMode.PASSTHROUGH - ): - raise NotImplementedError( - "ArrowSchemaConversionMode.passthrough is not implemented to work with DeltaSchema, skip passing a schema or pass an arrow schema." - ) - elif isinstance(schema, Schema): - if conversion_mode == ArrowSchemaConversionMode.LARGE: - schema = schema.to_pyarrow(as_large_types=True) - else: - schema = schema.to_pyarrow(as_large_types=False) - elif schema is None: - schema = data.schema + def struct_to_delta_dtype(dtype: DataType) -> DataType: + fields_cast = [f.with_type(dtype_to_delta_dtype(f.type)) for f in dtype.fields] + return DataType.struct(fields_cast) - return data, schema + return Arro3Schema([f.with_type(dtype_to_delta_dtype(f.type)) for f in schema]) # type: ignore[attr-defined] diff --git a/python/deltalake/writer/convert_to.py b/python/deltalake/writer/convert_to.py index ea53b947e7..592a052017 100644 --- a/python/deltalake/writer/convert_to.py +++ b/python/deltalake/writer/convert_to.py @@ -7,12 +7,11 @@ Literal, ) +from deltalake._internal import Schema from deltalake._internal import convert_to_deltalake as _convert_to_deltalake from deltalake.writer._utils import try_get_deltatable if TYPE_CHECKING: - import pyarrow as pa - from deltalake.transaction import ( CommitProperties, PostCommitHookProperties, @@ -22,7 +21,7 @@ def convert_to_deltalake( uri: str | Path, mode: Literal["error", "ignore"] = "error", - partition_by: pa.Schema | None = None, + partition_by: Schema | None = None, partition_strategy: Literal["hive"] | None = None, name: str | None = None, description: str | None = None, diff --git a/python/deltalake/writer/writer.py b/python/deltalake/writer/writer.py index 8dfb346ad0..ab021bcd8f 100644 --- a/python/deltalake/writer/writer.py +++ b/python/deltalake/writer/writer.py @@ -1,6 +1,6 @@ from __future__ import annotations -from collections.abc import Iterable, Mapping +from collections.abc import Mapping, Sequence from pathlib import Path from typing import ( TYPE_CHECKING, @@ -8,22 +8,14 @@ overload, ) -from pyarrow import RecordBatchReader +from arro3.core import RecordBatchReader +from arro3.core.types import ArrowArrayExportable, ArrowStreamExportable from deltalake._internal import write_to_deltalake as write_deltalake_rust -from deltalake.writer._conversion import ( - ArrowSchemaConversionMode, - ArrowStreamExportable, - _convert_data_and_schema, -) +from deltalake.writer._conversion import _convert_arro3_schema_to_delta from deltalake.writer._utils import try_get_table_and_table_uri if TYPE_CHECKING: - import pandas as pd - import pyarrow as pa - import pyarrow.dataset as ds - - from deltalake.schema import Schema as DeltaSchema from deltalake.table import DeltaTable, WriterProperties from deltalake.transaction import ( CommitProperties, @@ -34,15 +26,8 @@ @overload def write_deltalake( table_or_uri: str | Path | DeltaTable, - data: pd.DataFrame - | ds.Dataset - | pa.Table - | pa.RecordBatch - | Iterable[pa.RecordBatch] - | pa.RecordBatchReader - | ArrowStreamExportable, + data: ArrowStreamExportable | ArrowArrayExportable | Sequence[ArrowArrayExportable], *, - schema: pa.Schema | DeltaSchema | None = ..., partition_by: list[str] | str | None = ..., mode: Literal["error", "append", "ignore"] = ..., name: str | None = ..., @@ -60,15 +45,8 @@ def write_deltalake( @overload def write_deltalake( table_or_uri: str | Path | DeltaTable, - data: pd.DataFrame - | ds.Dataset - | pa.Table - | pa.RecordBatch - | Iterable[pa.RecordBatch] - | pa.RecordBatchReader - | ArrowStreamExportable, + data: ArrowStreamExportable | ArrowArrayExportable | Sequence[ArrowArrayExportable], *, - schema: pa.Schema | DeltaSchema | None = ..., partition_by: list[str] | str | None = ..., mode: Literal["overwrite"], name: str | None = ..., @@ -86,15 +64,8 @@ def write_deltalake( def write_deltalake( table_or_uri: str | Path | DeltaTable, - data: pd.DataFrame - | ds.Dataset - | pa.Table - | pa.RecordBatch - | Iterable[pa.RecordBatch] - | pa.RecordBatchReader - | ArrowStreamExportable, + data: ArrowStreamExportable | ArrowArrayExportable | Sequence[ArrowArrayExportable], *, - schema: pa.Schema | DeltaSchema | None = None, partition_by: list[str] | str | None = None, mode: Literal["error", "append", "overwrite", "ignore"] = "error", name: str | None = None, @@ -115,7 +86,6 @@ def write_deltalake( Args: table_or_uri: URI of a table or a DeltaTable object. data: Data to write. If passing iterable, the schema must also be given. - schema: Optional schema to write. partition_by: List of columns to partition the table by. Only required when creating a new table. mode: How to handle existing data. Default is to error if table already exists. @@ -145,15 +115,17 @@ def write_deltalake( if table is not None and mode == "ignore": return - data, schema = _convert_data_and_schema( - data=data, - schema=schema, - conversion_mode=ArrowSchemaConversionMode.PASSTHROUGH, - ) - data = RecordBatchReader.from_batches(schema, (batch for batch in data)) + if isinstance(data, Sequence): + data = RecordBatchReader.from_batches(data[0], data) # type: ignore + else: + data = RecordBatchReader.from_arrow(data) + + compatible_delta_schema = _convert_arro3_schema_to_delta(data.schema) + if table: table._table.write( data=data, + batch_schema=compatible_delta_schema, partition_by=partition_by, mode=mode, schema_mode=schema_mode, @@ -170,6 +142,7 @@ def write_deltalake( write_deltalake_rust( table_uri=table_uri, data=data, + batch_schema=compatible_delta_schema, partition_by=partition_by, mode=mode, schema_mode=schema_mode, diff --git a/python/docs/source/usage.rst b/python/docs/source/usage.rst index a6ef6b2569..2d9e794b89 100644 --- a/python/docs/source/usage.rst +++ b/python/docs/source/usage.rst @@ -265,11 +265,11 @@ from json, use `schema.Schema.from_json()`. >>> dt.schema().json() '{"type":"struct","fields":[{"name":"id","type":"long","nullable":true,"metadata":{}}]}' -Use `deltalake.schema.Schema.to_pyarrow()` to retrieve the PyArrow schema: +Use `deltalake.schema.Schema.to_arrow()` to retrieve the Arro3 schema: .. code-block:: python - >>> dt.schema().to_pyarrow() + >>> dt.schema().to_arrow() id: int64 @@ -309,6 +309,8 @@ provide the list of files that are part of the table and metadata about them, such as creation time, size, and statistics. You can get a data frame of the add actions data using :meth:`DeltaTable.get_add_actions`: + + .. code-block:: python >>> from deltalake import DeltaTable @@ -328,6 +330,7 @@ This works even with past versions of the table: 0 part-00000-c9b90f86-73e6-46c8-93ba-ff6bfaf892a... 440 2021-03-06 15:16:07 True 2 0 0 2 1 part-00001-911a94a2-43f6-4acb-8620-5e68c265498... 445 2021-03-06 15:16:07 True 3 0 2 4 + Querying Delta Tables --------------------- @@ -351,7 +354,7 @@ support filtering partitions and selecting particular columns. >>> from deltalake import DeltaTable >>> dt = DeltaTable("../rust/tests/data/delta-0.8.0-partitioned") - >>> dt.schema().to_pyarrow() + >>> dt.schema().to_arrow() value: string year: string month: string diff --git a/python/pyproject.toml b/python/pyproject.toml index 11e237eeaa..a7156dea0f 100644 --- a/python/pyproject.toml +++ b/python/pyproject.toml @@ -16,11 +16,12 @@ classifiers = [ "Programming Language :: Python :: 3.11", "Programming Language :: Python :: 3.12", ] -dependencies = ["pyarrow>=16"] dynamic = ["version"] +dependencies = ["arro3-core>=0.5.0", "deprecated>=1.2.18"] [project.optional-dependencies] pandas = ["pandas"] +pyarrow = ["pyarrow>=16"] [project.urls] documentation = "https://delta-io.github.io/delta-rs/" @@ -69,7 +70,7 @@ ignore = ["E501", "ANN401", "RUF040"] known-first-party = ["deltalake"] [tool.pytest.ini_options] -addopts = "-v -m 'not integration and not benchmark'" +addopts = "-v -m 'not integration and not benchmark and not no_pyarrow'" testpaths = ["tests", "deltalake"] markers = [ "integration: marks tests as integration tests (deselect with '-m \"not integration\"')", @@ -78,6 +79,8 @@ markers = [ "pandas: marks tests that require pandas", "polars: marks tests that require polars", "lakefs: marks tests that require lakefs", + "pyarrow: marks tests that require pyarrow", + "no_pyarrow: marks tests that test deltalake functionality without pyarrow", "unitycatalog_databricks: marks tests that require unitycatalog_databricks", "unitycatalog_oss: marks tests that require unitycatalog_oss", "pyspark: marks tests that require pyspark", @@ -95,7 +98,8 @@ dev = [ "pytest-timeout", "pytest-benchmark", "mypy==1.10.1", - "ruff==0.11.2", + "ruff>=0.11.2,<0.11.12", + "types-deprecated>=1.2.15.20250304", ] polars = ["polars==1.17.1"] lakefs = ["lakefs==0.8.0"] @@ -104,10 +108,7 @@ pyspark = [ "delta-spark", "numpy==1.26.4", # pyspark is not compatible with latest numpy ] -docs = [ - "sphinx<=4.5", - "sphinx-rtd-theme", -] +docs = ["sphinx<=4.5", "sphinx-rtd-theme"] other = [ "azure-storage-blob==12.20.0", "packaging>=20", diff --git a/python/src/datafusion.rs b/python/src/datafusion.rs new file mode 100644 index 0000000000..fddd3ad661 --- /dev/null +++ b/python/src/datafusion.rs @@ -0,0 +1,343 @@ +use std::any::Any; +use std::borrow::Cow; +use std::sync::Arc; + +use arrow_schema::Schema as ArrowSchema; +use datafusion_expr::utils::conjunction; +use datafusion_physical_expr::execution_props::ExecutionProps; +use datafusion_physical_expr::{create_physical_expr, PhysicalExpr}; +use datafusion_physical_plan::filter::FilterExec; +use datafusion_physical_plan::limit::GlobalLimitExec; +use datafusion_physical_plan::memory::{LazyBatchGenerator, LazyMemoryExec}; +use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::{ExecutionPlan, Statistics}; +use deltalake::datafusion::catalog::{Session, TableProvider}; +use deltalake::datafusion::common::{Column, DFSchema, Result as DataFusionResult}; +use deltalake::datafusion::datasource::TableType; +use deltalake::datafusion::logical_expr::{LogicalPlan, TableProviderFilterPushDown}; +use deltalake::datafusion::prelude::Expr; +use deltalake::{DeltaResult, DeltaTableError}; +use parking_lot::RwLock; + +#[derive(Debug)] +pub(crate) struct LazyTableProvider { + schema: Arc, + batches: Vec>>, +} + +impl LazyTableProvider { + /// Build a DeltaTableProvider + pub fn try_new( + schema: Arc, + batches: Vec>>, + ) -> DeltaResult { + Ok(LazyTableProvider { schema, batches }) + } +} + +#[async_trait::async_trait] +impl TableProvider for LazyTableProvider { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> Arc { + self.schema.clone() + } + + fn table_type(&self) -> TableType { + TableType::Base + } + + fn get_table_definition(&self) -> Option<&str> { + None + } + + fn get_logical_plan(&self) -> Option> { + None + } + + async fn scan( + &self, + _session: &dyn Session, + projection: Option<&Vec>, + filters: &[Expr], + limit: Option, + ) -> DataFusionResult> { + let mut plan: Arc = Arc::new(LazyMemoryExec::try_new( + self.schema(), + self.batches.clone(), + )?); + + let df_schema: DFSchema = plan.schema().try_into()?; + + if let Some(filter_expr) = conjunction(filters.iter().cloned()) { + let physical_expr = + create_physical_expr(&filter_expr, &df_schema, &ExecutionProps::new())?; + plan = Arc::new(FilterExec::try_new(physical_expr, plan)?); + } + + if let Some(projection) = projection { + let current_projection = (0..plan.schema().fields().len()).collect::>(); + if projection != ¤t_projection { + let execution_props = &ExecutionProps::new(); + let fields: DeltaResult, String)>> = projection + .iter() + .map(|i| { + let (table_ref, field) = df_schema.qualified_field(*i); + create_physical_expr( + &Expr::Column(Column::from((table_ref, field))), + &df_schema, + execution_props, + ) + .map(|expr| (expr, field.name().clone())) + .map_err(DeltaTableError::from) + }) + .collect(); + plan = Arc::new(ProjectionExec::try_new(fields?, plan)?); + } + } + + if let Some(limit) = limit { + plan = Arc::new(GlobalLimitExec::new(plan, 0, Some(limit))) + }; + + Ok(plan) + } + + fn supports_filters_pushdown( + &self, + filter: &[&Expr], + ) -> DataFusionResult> { + Ok(vec![TableProviderFilterPushDown::Inexact; filter.len()]) + } + + fn statistics(&self) -> Option { + None + } +} + +#[cfg(test)] +mod tests { + use super::*; + + use std::sync::Arc; + + use arrow_schema::{DataType, Field, Schema as ArrowSchema}; + use datafusion_expr::{col, lit}; + use datafusion_physical_plan::memory::LazyBatchGenerator; + use deltalake::arrow::array::{Int32Array, StringArray}; + use deltalake::arrow::record_batch::RecordBatch; + use deltalake::datafusion::common::Result as DataFusionResult; + use deltalake::datafusion::prelude::SessionContext; + use parking_lot::RwLock; + + // Import the LazyTableProvider + use crate::datafusion::LazyTableProvider; + + // A dummy LazyBatchGenerator implementation for testing + #[derive(Debug)] + struct TestBatchGenerator { + schema: Arc, + data: Vec, + current_index: usize, + } + + impl std::fmt::Display for TestBatchGenerator { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "TestBatchGenerator") + } + } + + impl TestBatchGenerator { + fn new(schema: Arc, data: Vec) -> Self { + Self { + schema, + data, + current_index: 0, + } + } + + // Helper to create a test batch generator with sample data + fn create_test_generator(schema: Arc) -> Arc> { + // Create sample data + let id_array = Int32Array::from(vec![1, 2, 3, 4, 5]); + let name_array = StringArray::from(vec!["Alice", "Bob", "Carol", "Dave", "Eve"]); + + let batch = RecordBatch::try_new( + schema.clone(), + vec![Arc::new(id_array), Arc::new(name_array)], + ) + .unwrap(); + + Arc::new(RwLock::new(TestBatchGenerator::new(schema, vec![batch]))) + } + } + + impl LazyBatchGenerator for TestBatchGenerator { + fn generate_next_batch(&mut self) -> DataFusionResult> { + if self.current_index < self.data.len() { + let batch = self.data[self.current_index].clone(); + self.current_index += 1; + Ok(Some(batch)) + } else { + Ok(None) + } + } + } + + #[tokio::test] + async fn test_lazy_table_provider_basic() { + // Create a schema + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + + // Create a test generator + let generator = TestBatchGenerator::create_test_generator(schema.clone()); + + // Create the LazyTableProvider + let provider = LazyTableProvider::try_new(schema.clone(), vec![generator]).unwrap(); + + // Check that the schema matches + assert_eq!(provider.schema(), schema); + + // Create a session context + let ctx = SessionContext::new(); + let session = ctx.state(); + + // Test basic scan without projections or filters + let plan = provider.scan(&session, None, &[], None).await.unwrap(); + assert_eq!(plan.schema().fields().len(), 2); + assert_eq!(plan.schema().field(0).name(), "id"); + assert_eq!(plan.schema().field(1).name(), "name"); + } + + #[tokio::test] + async fn test_lazy_table_provider_with_projection() { + // Create a schema + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + + // Create a test generator + let generator = TestBatchGenerator::create_test_generator(schema.clone()); + + // Create the LazyTableProvider + let provider = LazyTableProvider::try_new(schema, vec![generator]).unwrap(); + + // Create a session context + let ctx = SessionContext::new(); + let session = ctx.state(); + + // Test scanning with projection (only select the id column) + let projection = Some(vec![0]); + let plan = provider + .scan(&session, projection.as_ref(), &[], None) + .await + .unwrap(); + + // Verify the plan schema only includes the projected column + assert_eq!(plan.schema().fields().len(), 1); + assert_eq!(plan.schema().field(0).name(), "id"); + } + + #[tokio::test] + async fn test_lazy_table_provider_with_filter() { + // Create a schema + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + + // Create a test generator + let generator = TestBatchGenerator::create_test_generator(schema.clone()); + + // Create the LazyTableProvider + let provider = LazyTableProvider::try_new(schema, vec![generator]).unwrap(); + + // Create a session context + let ctx = SessionContext::new(); + let session = ctx.state(); + + // Test scanning with filter (id > 2) + let filter = vec![col("id").gt(lit(2))]; + let plan = provider.scan(&session, None, &filter, None).await.unwrap(); + + // The scan method should add a FilterExec to the plan + // We can verify this by checking the plan's children + assert!(plan.children().len() > 0); + } + + #[tokio::test] + async fn test_lazy_table_provider_with_limit() { + // Create a schema + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + + // Create a test generator + let generator = TestBatchGenerator::create_test_generator(schema.clone()); + + // Create the LazyTableProvider + let provider = LazyTableProvider::try_new(schema, vec![generator]).unwrap(); + + // Create a session context + let ctx = SessionContext::new(); + let session = ctx.state(); + + // Test scanning with limit + let limit = Some(3); + let plan = provider.scan(&session, None, &[], limit).await.unwrap(); + + // The plan should include a LimitExec + // We can verify this by checking that the plan type is correct + assert!(plan + .as_any() + .downcast_ref::() + .is_some()); + } + + #[tokio::test] + async fn test_lazy_table_provider_combined() { + // Create a schema + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + + // Create a test generator + let generator = TestBatchGenerator::create_test_generator(schema.clone()); + + // Create the LazyTableProvider + let provider = LazyTableProvider::try_new(schema, vec![generator]).unwrap(); + + // Create a session context + let ctx = SessionContext::new(); + let session = ctx.state(); + + // Test scanning with projection, filter, and limit combined + let projection = Some(vec![0]); // Only id column + let filter = vec![col("id").gt(lit(2))]; // id > 2 + let limit = Some(2); // Return only 2 rows + + let plan = provider + .scan(&session, projection.as_ref(), &filter, limit) + .await + .unwrap(); + + // Verify the plan schema only includes the projected column + assert_eq!(plan.schema().fields().len(), 1); + assert_eq!(plan.schema().field(0).name(), "id"); + + // The resulting plan should have a chain of operations: + // GlobalLimitExec -> ProjectionExec -> FilterExec -> LazyMemoryExec + assert!(plan + .as_any() + .downcast_ref::() + .is_some()); + } +} diff --git a/python/src/error.rs b/python/src/error.rs index 84809af19b..eaf50ce99c 100644 --- a/python/src/error.rs +++ b/python/src/error.rs @@ -1,6 +1,5 @@ use arrow_schema::ArrowError; use deltalake::datafusion::error::DataFusionError; -use deltalake::protocol::ProtocolError; use deltalake::{errors::DeltaTableError, ObjectStoreError}; use pyo3::exceptions::PyRuntimeError; use pyo3::exceptions::{ @@ -66,7 +65,7 @@ impl Display for DisplaySourceChain { for err_part in err_msg.split(": ").flat_map(|s| s.split("\ncaused by\n")) { if !err_part.is_empty() && !out_parts.contains(&err_part) - && !out_parts.iter().map(|p| p.contains(&err_part)).any(|v| v) + && !out_parts.iter().any(|p| p.contains(err_part)) { out_parts.push(err_part); } @@ -74,13 +73,12 @@ impl Display for DisplaySourceChain { } for (i, part) in out_parts.iter().enumerate() { if i == 0 { - write!(f, "{}\n", part)?; + writeln!(f, "{}", part)?; } else { - write!( + writeln!( f, - "{}\x1b[31m{}\x1b[0m {}\n", + "{}\x1b[31m↳\x1b[0m {}", " ".repeat(self.error_name.len() + ": ".len() + i), - "↳", part )?; } @@ -124,24 +122,6 @@ fn arrow_to_py(err: ArrowError) -> PyErr { } } -fn checkpoint_to_py(err: ProtocolError) -> PyErr { - match err { - ProtocolError::Arrow { source } => arrow_to_py(source), - ProtocolError::ObjectStore { source } => object_store_to_py(source), - ProtocolError::EndOfLog => DeltaProtocolError::new_err("End of log"), - ProtocolError::NoMetaData => DeltaProtocolError::new_err("Table metadata missing"), - ProtocolError::CheckpointNotFound => DeltaProtocolError::new_err(err.to_string()), - ProtocolError::InvalidField(err) => PyValueError::new_err(err), - ProtocolError::InvalidRow(err) => PyValueError::new_err(err), - ProtocolError::InvalidDeletionVectorStorageType(err) => PyValueError::new_err(err), - ProtocolError::SerializeOperation { source } => PyValueError::new_err(source.to_string()), - ProtocolError::ParquetParseError { source } => PyIOError::new_err(source.to_string()), - ProtocolError::IO { source } => PyIOError::new_err(source.to_string()), - ProtocolError::Generic(msg) => DeltaError::new_err(msg), - ProtocolError::Kernel { source } => DeltaError::new_err(source.to_string()), - } -} - fn datafusion_to_py(err: DataFusionError) -> PyErr { DeltaError::new_err(err.to_string()) } @@ -154,8 +134,6 @@ pub enum PythonError { ObjectStore(#[from] ObjectStoreError), #[error("Error in arrow")] Arrow(#[from] ArrowError), - #[error("Error in checkpoint")] - Protocol(#[from] ProtocolError), #[error("Error in data fusion")] DataFusion(#[from] DataFusionError), #[error("Lock acquisition error")] @@ -174,7 +152,6 @@ impl From for pyo3::PyErr { PythonError::DeltaTable(err) => inner_to_py_err(err), PythonError::ObjectStore(err) => object_store_to_py(err), PythonError::Arrow(err) => arrow_to_py(err), - PythonError::Protocol(err) => checkpoint_to_py(err), PythonError::DataFusion(err) => datafusion_to_py(err), PythonError::ThreadingError(err) => PyRuntimeError::new_err(err), } diff --git a/python/src/filesystem.rs b/python/src/filesystem.rs index b570f70a58..8554d188d7 100644 --- a/python/src/filesystem.rs +++ b/python/src/filesystem.rs @@ -1,8 +1,10 @@ use crate::error::PythonError; use crate::utils::{delete_dir, rt, walk_tree, warn}; use crate::RawDeltaTable; -use deltalake::storage::object_store::{MultipartUpload, PutPayloadMut}; -use deltalake::storage::{DynObjectStore, ListResult, ObjectStoreError, Path}; +use deltalake::logstore::object_store::{ + path::Path, DynObjectStore, Error as ObjectStoreError, ListResult, MultipartUpload, + PutPayloadMut, +}; use deltalake::DeltaTableBuilder; use parking_lot::Mutex; use pyo3::exceptions::{PyIOError, PyNotImplementedError, PyValueError}; @@ -461,15 +463,15 @@ impl ObjectInputFile { self.check_closed()?; let range = match nbytes { Some(len) => { - let end = i64::min(self.pos + len, self.content_length) as usize; + let end = i64::min(self.pos + len, self.content_length) as u64; std::ops::Range { - start: self.pos as usize, + start: self.pos as u64, end, } } _ => std::ops::Range { - start: self.pos as usize, - end: self.content_length as usize, + start: self.pos as u64, + end: self.content_length as u64, }, }; let nbytes = (range.end - range.start) as i64; diff --git a/python/src/lib.rs b/python/src/lib.rs index bd2aba0fc7..c73a0ba5a6 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -1,3 +1,4 @@ +mod datafusion; mod error; mod features; mod filesystem; @@ -8,34 +9,27 @@ mod schema; mod utils; mod writer; -use std::cmp::min; -use std::collections::{HashMap, HashSet}; -use std::ffi::CString; -use std::future::IntoFuture; -use std::str::FromStr; -use std::sync::{Arc, Mutex}; -use std::time; -use std::time::{SystemTime, UNIX_EPOCH}; - use arrow::pyarrow::PyArrowType; +use arrow_schema::SchemaRef; use chrono::{DateTime, Duration, FixedOffset, Utc}; use datafusion_ffi::table_provider::FFI_TableProvider; use delta_kernel::expressions::Scalar; use delta_kernel::schema::{MetadataValue, StructField}; -use deltalake::arrow::compute::concat_batches; -use deltalake::arrow::ffi_stream::{ArrowArrayStreamReader, FFI_ArrowArrayStream}; -use deltalake::arrow::record_batch::RecordBatch; use deltalake::arrow::{self, datatypes::Schema as ArrowSchema}; use deltalake::checkpoints::{cleanup_metadata, create_checkpoint}; use deltalake::datafusion::catalog::TableProvider; +use deltalake::datafusion::datasource::provider_as_source; +use deltalake::datafusion::logical_expr::LogicalPlanBuilder; use deltalake::datafusion::prelude::SessionContext; -use deltalake::delta_datafusion::{DeltaCdfTableProvider, DeltaDataChecker}; +use deltalake::delta_datafusion::DeltaCdfTableProvider; use deltalake::errors::DeltaTableError; +use deltalake::kernel::transaction::{CommitBuilder, CommitProperties, TableReference}; use deltalake::kernel::{ - scalars::ScalarExt, Action, Add, Invariant, LogicalFile, Remove, StructType, Transaction, + scalars::ScalarExt, Action, Add, LogicalFile, Remove, StructType, Transaction, }; use deltalake::lakefs::LakeFSCustomExecuteHandler; use deltalake::logstore::LogStoreRef; +use deltalake::logstore::{IORuntime, ObjectStoreRef}; use deltalake::operations::add_column::AddColumnBuilder; use deltalake::operations::add_feature::AddTableFeatureBuilder; use deltalake::operations::constraints::ConstraintBuilder; @@ -47,11 +41,12 @@ use deltalake::operations::load_cdf::CdfLoadBuilder; use deltalake::operations::optimize::{OptimizeBuilder, OptimizeType}; use deltalake::operations::restore::RestoreBuilder; use deltalake::operations::set_tbl_properties::SetTablePropertiesBuilder; -use deltalake::operations::transaction::{ - CommitBuilder, CommitProperties, TableReference, PROTOCOL, -}; use deltalake::operations::update::UpdateBuilder; -use deltalake::operations::vacuum::VacuumBuilder; +use deltalake::operations::update_field_metadata::UpdateFieldMetadataBuilder; +use deltalake::operations::update_table_metadata::{ + TableMetadataUpdate, UpdateTableMetadataBuilder, +}; +use deltalake::operations::vacuum::{VacuumBuilder, VacuumMode}; use deltalake::operations::write::WriteBuilder; use deltalake::operations::CustomExecuteHandler; use deltalake::parquet::basic::Compression; @@ -59,47 +54,44 @@ use deltalake::parquet::errors::ParquetError; use deltalake::parquet::file::properties::{EnabledStatistics, WriterProperties}; use deltalake::partitions::PartitionFilter; use deltalake::protocol::{DeltaOperation, SaveMode}; -use deltalake::storage::{IORuntime, ObjectStoreRef}; use deltalake::table::state::DeltaTableState; -use deltalake::{init_client_version, DeltaTableBuilder}; -use deltalake::{DeltaOps, DeltaResult}; -use error::DeltaError; -use reader::convert_stream_to_reader; -use tracing::log::*; - -use crate::writer::to_lazy_table; -use deltalake::datafusion::datasource::provider_as_source; -use deltalake::datafusion::logical_expr::LogicalPlanBuilder; - -use crate::error::DeltaProtocolError; -use crate::error::PythonError; -use crate::features::TableFeatures; -use crate::filesystem::FsConfig; -use crate::merge::PyMergeBuilder; -use crate::query::PyQueryBuilder; -use crate::schema::{schema_to_pyobject, Field}; -use crate::utils::rt; -use deltalake::operations::update_field_metadata::UpdateFieldMetadataBuilder; +use deltalake::{init_client_version, DeltaOps, DeltaResult, DeltaTableBuilder}; use pyo3::exceptions::{PyRuntimeError, PyValueError}; use pyo3::pybacked::PyBackedStr; use pyo3::types::{PyCapsule, PyDict, PyFrozenSet}; use pyo3::{prelude::*, IntoPyObjectExt}; +use pyo3_arrow::export::{Arro3RecordBatch, Arro3RecordBatchReader}; +use pyo3_arrow::{PyRecordBatchReader, PySchema as PyArrowSchema}; +use schema::PySchema; use serde_json::{Map, Value}; +use std::cmp::min; +use std::collections::{HashMap, HashSet}; +use std::ffi::CString; +use std::future::IntoFuture; +use std::str::FromStr; +use std::sync::{Arc, Mutex}; +use std::time; +use std::time::{SystemTime, UNIX_EPOCH}; use uuid::Uuid; +use writer::maybe_lazy_cast_reader; -#[cfg(all(target_family = "unix", not(target_os = "emscripten")))] -use jemallocator::Jemalloc; - -#[cfg(any(not(target_family = "unix"), target_os = "emscripten"))] -use mimalloc::MiMalloc; +use crate::error::{DeltaError, DeltaProtocolError, PythonError}; +use crate::features::TableFeatures; +use crate::filesystem::FsConfig; +use crate::merge::PyMergeBuilder; +use crate::query::PyQueryBuilder; +use crate::reader::convert_stream_to_reader; +use crate::schema::{schema_to_pyobject, Field}; +use crate::utils::rt; +use crate::writer::to_lazy_table; #[global_allocator] #[cfg(all(target_family = "unix", not(target_os = "emscripten")))] -static ALLOC: Jemalloc = Jemalloc; +static ALLOC: jemallocator::Jemalloc = jemallocator::Jemalloc; #[global_allocator] #[cfg(any(not(target_family = "unix"), target_os = "emscripten"))] -static ALLOC: MiMalloc = MiMalloc; +static ALLOC: mimalloc::MiMalloc = mimalloc::MiMalloc; #[derive(FromPyObject)] enum PartitionFilterValue { @@ -154,7 +146,7 @@ impl RawDeltaTable { } fn object_store(&self) -> PyResult { - self.with_table(|t| Ok(t.object_store().clone())) + self.with_table(|t| Ok(t.log_store().object_store(None).clone())) } fn cloned_state(&self) -> PyResult { @@ -246,11 +238,11 @@ impl RawDeltaTable { self.with_table(|t| Ok(t.table_uri())) } - pub fn version(&self) -> PyResult { + pub fn version(&self) -> PyResult> { self.with_table(|t| Ok(t.version())) } - pub fn has_files(&self) -> PyResult { + pub(crate) fn has_files(&self) -> PyResult { self.with_table(|t| Ok(t.config.require_files)) } @@ -308,21 +300,6 @@ impl RawDeltaTable { )) } - pub fn check_can_write_timestamp_ntz(&self, schema: PyArrowType) -> PyResult<()> { - let schema: StructType = (&schema.0).try_into().map_err(PythonError::from)?; - // Need to unlock to access the shared reference to &DeltaTableState - match self._table.lock() { - Ok(table) => Ok(PROTOCOL - .check_can_write_timestamp_ntz( - table.snapshot().map_err(PythonError::from)?, - &schema, - ) - .map_err(|e| DeltaTableError::Generic(e.to_string())) - .map_err(PythonError::from)?), - Err(e) => Err(PyRuntimeError::new_err(e.to_string())), - } - } - /// Load the internal [RawDeltaTable] with the table state from the specified `version` /// /// This will acquire the internal lock since it is a mutating operation! @@ -360,23 +337,7 @@ impl RawDeltaTable { }) } - pub fn get_earliest_version(&self, py: Python) -> PyResult { - py.allow_threads(|| { - #[allow(clippy::await_holding_lock)] - rt().block_on(async { - match self._table.lock() { - Ok(table) => table - .get_earliest_version() - .await - .map_err(PythonError::from) - .map_err(PyErr::from), - Err(e) => Err(PyRuntimeError::new_err(e.to_string())), - } - }) - }) - } - - pub fn get_num_index_cols(&self) -> PyResult { + fn get_num_index_cols(&self) -> PyResult { self.with_table(|t| { Ok(t.snapshot() .map_err(PythonError::from)? @@ -385,7 +346,7 @@ impl RawDeltaTable { }) } - pub fn get_stats_columns(&self) -> PyResult>> { + fn get_stats_columns(&self) -> PyResult>> { self.with_table(|t| { Ok(t.snapshot() .map_err(PythonError::from)? @@ -489,7 +450,8 @@ impl RawDeltaTable { /// Run the Vacuum command on the Delta Table: list and delete files no longer referenced /// by the Delta table and are older than the retention threshold. - #[pyo3(signature = (dry_run, retention_hours = None, enforce_retention_duration = true, commit_properties=None, post_commithook_properties=None))] + #[pyo3(signature = (dry_run, retention_hours = None, enforce_retention_duration = true, commit_properties=None, post_commithook_properties=None, full = false))] + #[allow(clippy::too_many_arguments)] pub fn vacuum( &self, py: Python, @@ -498,6 +460,7 @@ impl RawDeltaTable { enforce_retention_duration: bool, commit_properties: Option, post_commithook_properties: Option, + full: bool, ) -> PyResult> { let (table, metrics) = py.allow_threads(|| { let snapshot = match self._table.lock() { @@ -511,6 +474,11 @@ impl RawDeltaTable { let mut cmd = VacuumBuilder::new(self.log_store()?, snapshot) .with_enforce_retention_duration(enforce_retention_duration) .with_dry_run(dry_run); + + if full { + cmd = cmd.with_mode(VacuumMode::Full); + } + if let Some(retention_period) = retention_hours { cmd = cmd.with_retention_period(Duration::hours(retention_period as i64)); } @@ -854,7 +822,7 @@ impl RawDeltaTable { columns: Option>, predicate: Option, allow_out_of_range: bool, - ) -> PyResult> { + ) -> PyResult { let ctx = SessionContext::new(); let mut cdf_read = CdfLoadBuilder::new(self.log_store()?, self.cloned_state()?); @@ -905,15 +873,15 @@ impl RawDeltaTable { .map_err(PythonError::from)?; py.allow_threads(|| { - let ffi_stream = FFI_ArrowArrayStream::new(convert_stream_to_reader(stream)); - let reader = ArrowArrayStreamReader::try_new(ffi_stream).unwrap(); - Ok(PyArrowType(reader)) + let stream = convert_stream_to_reader(stream); + Ok(stream.into()) }) } #[allow(clippy::too_many_arguments)] #[pyo3(signature = ( source, + batch_schema, predicate, source_alias = None, target_alias = None, @@ -927,7 +895,8 @@ impl RawDeltaTable { pub fn create_merge_builder( &self, py: Python, - source: PyArrowType, + source: PyRecordBatchReader, + batch_schema: PyArrowSchema, predicate: String, source_alias: Option, target_alias: Option, @@ -949,7 +918,8 @@ impl RawDeltaTable { Ok(PyMergeBuilder::new( self.log_store()?, self.cloned_state()?, - source.0, + source, + batch_schema, predicate, source_alias, target_alias, @@ -1059,7 +1029,7 @@ impl RawDeltaTable { pub fn dataset_partitions<'py>( &self, py: Python<'py>, - schema: PyArrowType, + schema: PyArrowSchema, partition_filters: Option>, ) -> PyResult>)>> { let path_set = match partition_filters { @@ -1071,19 +1041,20 @@ impl RawDeltaTable { let stats_cols = self.get_stats_columns()?; let num_index_cols = self.get_num_index_cols()?; + let schema = schema.into_inner(); + let inclusion_stats_cols = if let Some(stats_cols) = stats_cols { stats_cols } else if num_index_cols == -1 { schema - .0 .fields() .iter() .map(|v| v.name().clone()) .collect::>() } else if num_index_cols >= 0 { let mut fields = vec![]; - for idx in 0..(min(num_index_cols as usize, schema.0.fields.len())) { - fields.push(schema.0.field(idx).name().clone()) + for idx in 0..(min(num_index_cols as usize, schema.fields.len())) { + fields.push(schema.field(idx).name().clone()) } fields } else { @@ -1213,16 +1184,15 @@ impl RawDeltaTable { add_actions: Vec, mode: &str, partition_by: Vec, - schema: PyArrowType, + schema: PyRef, partitions_filters: Option>, commit_properties: Option, post_commithook_properties: Option, ) -> PyResult<()> { + let schema = schema.as_ref().inner_type.clone(); py.allow_threads(|| { let mode = mode.parse().map_err(PythonError::from)?; - let schema: StructType = (&schema.0).try_into().map_err(PythonError::from)?; - let existing_schema = self.with_table(|t| { t.get_schema() .cloned() @@ -1336,14 +1306,6 @@ impl RawDeltaTable { }) } - pub fn get_py_storage_backend(&self) -> PyResult { - Ok(filesystem::DeltaFileSystemHandler { - inner: self.object_store()?, - config: self._config.clone(), - known_sizes: None, - }) - } - pub fn create_checkpoint(&self, py: Python) -> PyResult<()> { py.allow_threads(|| { let operation_id = Uuid::new_v4(); @@ -1416,10 +1378,9 @@ impl RawDeltaTable { let new_state = if result > 0 { Some( DeltaTableState::try_new( - &table.state.clone().unwrap().snapshot().table_root(), - table.object_store(), + &table.log_store(), table.config.clone(), - Some(table.version()), + table.version(), ) .await .map_err(PythonError::from)?, @@ -1453,16 +1414,18 @@ impl RawDeltaTable { Ok(()) } - pub fn get_add_actions(&self, flatten: bool) -> PyResult> { + pub fn get_add_actions(&self, flatten: bool) -> PyResult { + // replace with Arro3RecordBatch once new release is done for arro3.core if !self.has_files()? { return Err(DeltaError::new_err("Table is instantiated without files.")); } - Ok(PyArrowType(self.with_table(|t| { + let batch = self.with_table(|t| { Ok(t.snapshot() .map_err(PythonError::from)? .add_actions_table(flatten) .map_err(PythonError::from)?) - })?)) + })?; + Ok(batch.into()) } pub fn get_add_file_sizes(&self) -> PyResult> { @@ -1540,6 +1503,62 @@ impl RawDeltaTable { Ok(()) } + #[pyo3(signature = (name, commit_properties=None))] + pub fn set_table_name( + &self, + name: String, + commit_properties: Option, + ) -> PyResult<()> { + let update = TableMetadataUpdate { + name: Some(name), + description: None, + }; + let mut cmd = UpdateTableMetadataBuilder::new(self.log_store()?, self.cloned_state()?) + .with_update(update); + + if let Some(commit_properties) = maybe_create_commit_properties(commit_properties, None) { + cmd = cmd.with_commit_properties(commit_properties); + } + + if self.log_store()?.name() == "LakeFSLogStore" { + cmd = cmd.with_custom_execute_handler(Arc::new(LakeFSCustomExecuteHandler {})); + } + + let table = rt() + .block_on(cmd.into_future()) + .map_err(PythonError::from)?; + self.set_state(table.state)?; + Ok(()) + } + + #[pyo3(signature = (description, commit_properties=None))] + pub fn set_table_description( + &self, + description: String, + commit_properties: Option, + ) -> PyResult<()> { + let update = TableMetadataUpdate { + name: None, + description: Some(description), + }; + let mut cmd = UpdateTableMetadataBuilder::new(self.log_store()?, self.cloned_state()?) + .with_update(update); + + if let Some(commit_properties) = maybe_create_commit_properties(commit_properties, None) { + cmd = cmd.with_commit_properties(commit_properties); + } + + if self.log_store()?.name() == "LakeFSLogStore" { + cmd = cmd.with_custom_execute_handler(Arc::new(LakeFSCustomExecuteHandler {})); + } + + let table = rt() + .block_on(cmd.into_future()) + .map_err(PythonError::from)?; + self.set_state(table.state)?; + Ok(()) + } + /// Execute the File System Check command (FSCK) on the delta table: removes old reference to files that /// have been deleted or are malformed #[pyo3(signature = (dry_run = true, commit_properties = None, post_commithook_properties=None))] @@ -1569,19 +1588,16 @@ impl RawDeltaTable { Ok(serde_json::to_string(&metrics).unwrap()) } - pub fn transaction_versions(&self) -> HashMap { - let version = self.with_table(|t| Ok(t.get_app_transaction_version())); - - match version { - Ok(version) => version - .into_iter() - .map(|(app_id, transaction)| (app_id, PyTransaction::from(transaction))) - .collect(), - Err(e) => { - warn!("Cannot fetch transaction version due to {e:?}"); - HashMap::default() - } - } + /// Get the latest transaction version for the given application ID. + /// + /// Returns `None` if the application ID is not found. + pub fn transaction_version(&self, app_id: String) -> PyResult> { + // NOTE: this will simplify once we have moved logstore onto state. + let log_store = self.log_store()?; + let snapshot = self.with_table(|t| Ok(t.snapshot().map_err(PythonError::from)?.clone()))?; + Ok(rt() + .block_on(snapshot.transaction_version(log_store.as_ref(), app_id)) + .map_err(PythonError::from)?) } #[pyo3(signature = (field_name, metadata, commit_properties=None, post_commithook_properties=None))] @@ -1622,11 +1638,12 @@ impl RawDeltaTable { } #[allow(clippy::too_many_arguments)] - #[pyo3(signature = (data, mode, schema_mode=None, partition_by=None, predicate=None, target_file_size=None, name=None, description=None, configuration=None, writer_properties=None, commit_properties=None, post_commithook_properties=None))] + #[pyo3(signature = (data, batch_schema, mode, schema_mode=None, partition_by=None, predicate=None, target_file_size=None, name=None, description=None, configuration=None, writer_properties=None, commit_properties=None, post_commithook_properties=None))] fn write( &mut self, py: Python, - data: PyArrowType, + data: PyRecordBatchReader, + batch_schema: PyArrowSchema, mode: String, schema_mode: Option, partition_by: Option>, @@ -1650,7 +1667,11 @@ impl RawDeltaTable { ) .with_save_mode(save_mode); - let table_provider = to_lazy_table(data.0).map_err(PythonError::from)?; + let table_provider = to_lazy_table(maybe_lazy_cast_reader( + data.into_reader()?, + batch_schema.into_inner(), + )) + .map_err(PythonError::from)?; let plan = LogicalPlanBuilder::scan("source", provider_as_source(table_provider), None) .map_err(PythonError::from)? @@ -1930,7 +1951,7 @@ fn scalar_to_py<'py>(value: &Scalar, py_date: &Bound<'py, PyAny>) -> PyResult value.serialize().into_py_any(py)?, + Decimal(_) => value.serialize().into_py_any(py)?, Struct(data) => { let py_struct = PyDict::new(py); for (field, value) in data.fields().iter().zip(data.values().iter()) { @@ -1939,6 +1960,13 @@ fn scalar_to_py<'py>(value: &Scalar, py_date: &Bound<'py, PyAny>) -> PyResult todo!("how should this be converted!"), + Map(map) => { + let py_map = PyDict::new(py); + for (key, value) in map.pairs() { + py_map.set_item(scalar_to_py(key, py_date)?, scalar_to_py(value, py_date)?)?; + } + py_map.into_py_any(py)? + } }; Ok(val.into_bound(py)) @@ -1958,7 +1986,7 @@ fn scalar_to_py<'py>(value: &Scalar, py_date: &Bound<'py, PyAny>) -> PyResult( py: Python<'py>, - schema: &PyArrowType, + schema: &SchemaRef, stats_columns: &[String], file_info: LogicalFile<'_>, ) -> PyResult>> { @@ -1987,7 +2015,7 @@ fn filestats_to_expression_next<'py>( if !value.is_null() { // value is a string, but needs to be parsed into appropriate type let converted_value = - cast_to_type(&column, &scalar_to_py(value, &py_date)?, &schema.0)?; + cast_to_type(&column, &scalar_to_py(value, &py_date)?, schema)?; expressions.push( py_field .call1((&column,))? @@ -2027,7 +2055,7 @@ fn filestats_to_expression_next<'py>( Scalar::Struct(_) => {} _ => { let maybe_minimum = - cast_to_type(field.name(), &scalar_to_py(value, &py_date)?, &schema.0); + cast_to_type(field.name(), &scalar_to_py(value, &py_date)?, schema); if let Ok(minimum) = maybe_minimum { let field_expr = py_field.call1((field.name(),))?; let expr = field_expr.call_method1("__ge__", (minimum,)); @@ -2056,7 +2084,7 @@ fn filestats_to_expression_next<'py>( Scalar::Struct(_) => {} _ => { let maybe_maximum = - cast_to_type(field.name(), &scalar_to_py(value, &py_date)?, &schema.0); + cast_to_type(field.name(), &scalar_to_py(value, &py_date)?, schema); if let Ok(maximum) = maybe_maximum { let field_expr = py_field.call1((field.name(),))?; let expr = field_expr.call_method1("__le__", (maximum,)); @@ -2091,21 +2119,6 @@ fn rust_core_version() -> &'static str { deltalake::crate_version() } -#[pyfunction] -fn batch_distinct(batch: PyArrowType) -> PyResult> { - let ctx = SessionContext::new(); - let schema = batch.0.schema(); - ctx.register_batch("batch", batch.0) - .map_err(|err| PyRuntimeError::new_err(err.to_string()))?; - let batches = rt() - .block_on(async { ctx.table("batch").await?.distinct()?.collect().await }) - .map_err(|err| PyRuntimeError::new_err(err.to_string()))?; - - Ok(PyArrowType( - concat_batches(&schema, &batches).map_err(PythonError::from)?, - )) -} - fn current_timestamp() -> i64 { let start = SystemTime::now(); let since_the_epoch = start @@ -2133,7 +2146,6 @@ impl From<&PyAddAction> for Add { modification_time: action.modification_time, data_change: action.data_change, stats: action.stats.clone(), - stats_parsed: None, tags: None, deletion_vector: None, base_row_id: None, @@ -2239,11 +2251,12 @@ pub struct PyCommitProperties { #[pyfunction] #[allow(clippy::too_many_arguments)] -#[pyo3(signature = (table_uri, data, mode, schema_mode=None, partition_by=None, predicate=None, target_file_size=None, name=None, description=None, configuration=None, storage_options=None, writer_properties=None, commit_properties=None, post_commithook_properties=None))] +#[pyo3(signature = (table_uri, data, batch_schema, mode, schema_mode=None, partition_by=None, predicate=None, target_file_size=None, name=None, description=None, configuration=None, storage_options=None, writer_properties=None, commit_properties=None, post_commithook_properties=None))] fn write_to_deltalake( py: Python, table_uri: String, - data: PyArrowType, + data: PyRecordBatchReader, + batch_schema: PyArrowSchema, mode: String, schema_mode: Option, partition_by: Option>, @@ -2279,6 +2292,7 @@ fn write_to_deltalake( raw_table.map_err(PythonError::from)?.write( py, data, + batch_schema, mode, schema_mode, partition_by, @@ -2299,7 +2313,7 @@ fn write_to_deltalake( fn create_deltalake( py: Python, table_uri: String, - schema: PyArrowType, + schema: PyRef, partition_by: Vec, mode: String, raise_if_key_not_exists: bool, @@ -2310,6 +2324,7 @@ fn create_deltalake( commit_properties: Option, post_commithook_properties: Option, ) -> PyResult<()> { + let schema = schema.as_ref().inner_type.clone(); py.allow_threads(|| { let table = DeltaTableBuilder::from_uri(table_uri.clone()) .with_storage_options(storage_options.unwrap_or_default()) @@ -2317,7 +2332,6 @@ fn create_deltalake( .map_err(PythonError::from)?; let mode = mode.parse().map_err(PythonError::from)?; - let schema: StructType = (&schema.0).try_into().map_err(PythonError::from)?; let use_lakefs_handler = table.log_store().name() == "LakeFSLogStore"; @@ -2363,7 +2377,7 @@ fn create_deltalake( fn create_table_with_add_actions( py: Python, table_uri: String, - schema: PyArrowType, + schema: PyRef, add_actions: Vec, mode: &str, partition_by: Vec, @@ -2374,14 +2388,14 @@ fn create_table_with_add_actions( commit_properties: Option, post_commithook_properties: Option, ) -> PyResult<()> { + let schema = schema.as_ref().inner_type.clone(); + py.allow_threads(|| { let table = DeltaTableBuilder::from_uri(table_uri.clone()) .with_storage_options(storage_options.unwrap_or_default()) .build() .map_err(PythonError::from)?; - let schema: StructType = (&schema.0).try_into().map_err(PythonError::from)?; - let use_lakefs_handler = table.log_store().name() == "LakeFSLogStore"; let mut builder = DeltaOps(table) @@ -2426,7 +2440,7 @@ fn create_table_with_add_actions( fn convert_to_deltalake( py: Python, uri: String, - partition_schema: Option>, + partition_schema: Option>, partition_strategy: Option, name: Option, description: Option, @@ -2435,12 +2449,12 @@ fn convert_to_deltalake( commit_properties: Option, post_commithook_properties: Option, ) -> PyResult<()> { + let partition_schema = partition_schema.map(|s| s.as_ref().inner_type.clone()); py.allow_threads(|| { let mut builder = ConvertToDeltaBuilder::new().with_location(uri.clone()); if let Some(part_schema) = partition_schema { - let schema: StructType = (&part_schema.0).try_into().map_err(PythonError::from)?; - builder = builder.with_partition_schema(schema.fields().cloned()); + builder = builder.with_partition_schema(part_schema.fields().cloned()); } if let Some(partition_strategy) = &partition_strategy { @@ -2499,38 +2513,6 @@ fn get_num_idx_cols_and_stats_columns( } } -#[pyclass(name = "DeltaDataChecker", module = "deltalake._internal")] -struct PyDeltaDataChecker { - inner: DeltaDataChecker, - rt: tokio::runtime::Runtime, -} - -#[pymethods] -impl PyDeltaDataChecker { - #[new] - #[pyo3(signature = (invariants))] - fn new(invariants: Vec<(String, String)>) -> Self { - let invariants: Vec = invariants - .into_iter() - .map(|(field_name, invariant_sql)| Invariant { - field_name, - invariant_sql, - }) - .collect(); - Self { - inner: DeltaDataChecker::new_with_invariants(invariants), - rt: tokio::runtime::Runtime::new().unwrap(), - } - } - - fn check_batch(&self, batch: PyArrowType) -> PyResult<()> { - Ok(self - .rt - .block_on(async { self.inner.check_batch(&batch.0).await }) - .map_err(PythonError::from)?) - } -} - #[pymodule] // module name need to match project name fn _internal(m: &Bound<'_, PyModule>) -> PyResult<()> { @@ -2559,7 +2541,6 @@ fn _internal(m: &Bound<'_, PyModule>) -> PyResult<()> { m.add_function(pyo3::wrap_pyfunction!(create_table_with_add_actions, m)?)?; m.add_function(pyo3::wrap_pyfunction!(write_to_deltalake, m)?)?; m.add_function(pyo3::wrap_pyfunction!(convert_to_deltalake, m)?)?; - m.add_function(pyo3::wrap_pyfunction!(batch_distinct, m)?)?; m.add_function(pyo3::wrap_pyfunction!( get_num_idx_cols_and_stats_columns, m @@ -2568,7 +2549,6 @@ fn _internal(m: &Bound<'_, PyModule>) -> PyResult<()> { m.add_class::()?; m.add_class::()?; m.add_class::()?; - m.add_class::()?; m.add_class::()?; // There are issues with submodules, so we will expose them flat for now // See also: https://github.com/PyO3/pyo3/issues/759 diff --git a/python/src/merge.rs b/python/src/merge.rs index 4d57423eb1..eeb5197cbe 100644 --- a/python/src/merge.rs +++ b/python/src/merge.rs @@ -1,26 +1,24 @@ -use deltalake::arrow::array::RecordBatchReader; use deltalake::arrow::datatypes::Schema as ArrowSchema; -use deltalake::arrow::ffi_stream::ArrowArrayStreamReader; -use deltalake::arrow::pyarrow::IntoPyArrow; use deltalake::datafusion::catalog::TableProvider; use deltalake::datafusion::datasource::MemTable; use deltalake::datafusion::physical_plan::memory::LazyBatchGenerator; use deltalake::datafusion::prelude::SessionContext; -use deltalake::delta_datafusion::LazyTableProvider; use deltalake::logstore::LogStoreRef; use deltalake::operations::merge::MergeBuilder; use deltalake::operations::CustomExecuteHandler; use deltalake::table::state::DeltaTableState; -use deltalake::{DeltaResult, DeltaTable}; +use deltalake::{DeltaResult, DeltaTable, DeltaTableError}; use parking_lot::RwLock; use pyo3::prelude::*; +use pyo3_arrow::{PyRecordBatchReader, PySchema as PyArrowSchema}; use std::collections::HashMap; -use std::fmt::{self}; use std::future::IntoFuture; -use std::sync::{Arc, Mutex}; +use std::sync::Arc; +use crate::datafusion::LazyTableProvider; use crate::error::PythonError; use crate::utils::rt; +use crate::writer::{maybe_lazy_cast_reader, ArrowStreamBatchGenerator}; use crate::{ maybe_create_commit_properties, set_writer_properties, PyCommitProperties, PyPostCommitHookProperties, PyWriterProperties, @@ -37,53 +35,14 @@ pub(crate) struct PyMergeBuilder { merge_schema: bool, arrow_schema: Arc, } -#[derive(Debug)] -struct ArrowStreamBatchGenerator { - pub array_stream: Arc>, -} - -impl fmt::Display for ArrowStreamBatchGenerator { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!( - f, - "ArrowStreamBatchGenerator {{ array_stream: {:?} }}", - self.array_stream - ) - } -} - -impl ArrowStreamBatchGenerator { - fn new(array_stream: Arc>) -> Self { - Self { array_stream } - } -} - -impl LazyBatchGenerator for ArrowStreamBatchGenerator { - fn generate_next_batch( - &mut self, - ) -> deltalake::datafusion::error::Result> { - let mut stream_reader = self.array_stream.lock().map_err(|_| { - deltalake::datafusion::error::DataFusionError::Execution( - "Failed to lock the ArrowArrayStreamReader".to_string(), - ) - })?; - - match stream_reader.next() { - Some(Ok(record_batch)) => Ok(Some(record_batch)), - Some(Err(err)) => Err(deltalake::datafusion::error::DataFusionError::ArrowError( - err, None, - )), - None => Ok(None), // End of stream - } - } -} impl PyMergeBuilder { #[allow(clippy::too_many_arguments)] pub fn new( log_store: LogStoreRef, snapshot: DeltaTableState, - source: ArrowArrayStreamReader, + source: PyRecordBatchReader, + batch_schema: PyArrowSchema, predicate: String, source_alias: Option, target_alias: Option, @@ -96,12 +55,17 @@ impl PyMergeBuilder { custom_execute_handler: Option>, ) -> DeltaResult { let ctx = SessionContext::new(); + + let source = source + .into_reader() + .map_err(|e| DeltaTableError::generic(e.to_string()))?; + + let source = maybe_lazy_cast_reader(source, batch_schema.into_inner()); let schema = source.schema(); let source_df = if streamed_exec { - let arrow_stream: Arc> = Arc::new(Mutex::new(source)); let arrow_stream_batch_generator: Arc> = - Arc::new(RwLock::new(ArrowStreamBatchGenerator::new(arrow_stream))); + Arc::new(RwLock::new(ArrowStreamBatchGenerator::new(source))); let table_provider: Arc = Arc::new(LazyTableProvider::try_new( schema.clone(), @@ -161,8 +125,8 @@ impl PyMergeBuilder { #[pymethods] impl PyMergeBuilder { #[getter] - fn get_arrow_schema(&self, py: Python) -> PyResult { - ::clone(&self.arrow_schema).into_pyarrow(py) + fn get_arrow_schema(&self) -> PyArrowSchema { + PyArrowSchema::new(self.arrow_schema.clone()) } #[pyo3(signature=( diff --git a/python/src/query.rs b/python/src/query.rs index 3a5d979c8e..0f9f0ed09d 100644 --- a/python/src/query.rs +++ b/python/src/query.rs @@ -1,13 +1,15 @@ use std::sync::Arc; use deltalake::{ - arrow::pyarrow::ToPyArrow, datafusion::prelude::SessionContext, - delta_datafusion::{DeltaScanConfigBuilder, DeltaSessionConfig, DeltaTableProvider}, + delta_datafusion::{ + DataFusionMixins, DeltaScanConfigBuilder, DeltaSessionConfig, DeltaTableProvider, + }, }; use pyo3::prelude::*; +use pyo3_arrow::PyRecordBatchReader; -use crate::{error::PythonError, utils::rt, RawDeltaTable}; +use crate::{convert_stream_to_reader, error::PythonError, utils::rt, RawDeltaTable}; /// PyQueryBuilder supports the _experimental_ `QueryBuilder` Python interface which allows users /// to take advantage of the [Apache DataFusion](https://datafusion.apache.org) engine already @@ -32,13 +34,14 @@ impl PyQueryBuilder { /// Register the given [RawDeltaTable] into the [SessionContext] using the provided /// `table_name` /// - /// Once called, the provided `delta_table` will be referencable in SQL queries so long as + /// Once called, the provided `delta_table` will be referenceable in SQL queries so long as /// another table of the same name is not registered over it. pub fn register(&self, table_name: &str, delta_table: &RawDeltaTable) -> PyResult<()> { let snapshot = delta_table.cloned_state()?; let log_store = delta_table.log_store()?; let scan_config = DeltaScanConfigBuilder::default() + .with_schema(snapshot.input_schema().map_err(PythonError::from)?) .build(&snapshot) .map_err(PythonError::from)?; @@ -59,15 +62,16 @@ impl PyQueryBuilder { /// **NOTE:** Since this function returns a materialized Python list of `RecordBatch` /// instances, it may result unexpected memory consumption for queries which return large data /// sets. - pub fn execute(&self, py: Python, sql: &str) -> PyResult { - let batches = py.allow_threads(|| { + pub fn execute(&self, py: Python, sql: &str) -> PyResult { + let stream = py.allow_threads(|| { rt().block_on(async { let df = self.ctx.sql(sql).await?; - df.collect().await + df.execute_stream().await }) .map_err(PythonError::from) })?; - batches.to_pyarrow(py) + let stream = convert_stream_to_reader(stream); + Ok(PyRecordBatchReader::new(stream)) } } diff --git a/python/src/schema.rs b/python/src/schema.rs index b93fdfc219..91681e0d96 100644 --- a/python/src/schema.rs +++ b/python/src/schema.rs @@ -1,18 +1,26 @@ extern crate pyo3; +use delta_kernel::engine::arrow_conversion::{TryIntoArrow, TryIntoKernel}; use deltalake::arrow::datatypes::{ DataType as ArrowDataType, Field as ArrowField, FieldRef as ArrowFieldRef, Schema as ArrowSchema, }; use deltalake::arrow::error::ArrowError; -use deltalake::arrow::pyarrow::PyArrowType; use deltalake::kernel::{ ArrayType as DeltaArrayType, DataType, MapType as DeltaMapType, MetadataValue, - PrimitiveType as DeltaPrimitve, StructField, StructType as DeltaStructType, StructTypeExt, + PrimitiveType as DeltaPrimitive, StructField, StructType as DeltaStructType, StructTypeExt, }; use pyo3::exceptions::{PyException, PyNotImplementedError, PyTypeError, PyValueError}; +use pyo3::types::PyCapsule; use pyo3::{prelude::*, IntoPyObjectExt}; +use pyo3_arrow::error::PyArrowResult; +use pyo3_arrow::export::{Arro3DataType, Arro3Field, Arro3Schema}; +use pyo3_arrow::ffi::to_schema_pycapsule; +use pyo3_arrow::PyDataType; +use pyo3_arrow::PyField; +use pyo3_arrow::PySchema as PyArrow3Schema; use std::collections::HashMap; +use std::sync::Arc; use crate::utils::warn; @@ -67,7 +75,7 @@ fn python_type_to_schema(ob: &Bound<'_, PyAny>) -> PyResult { #[pyclass(module = "deltalake._internal")] #[derive(Clone)] pub struct PrimitiveType { - inner_type: DeltaPrimitve, + inner_type: DeltaPrimitive, } impl TryFrom for PrimitiveType { @@ -85,7 +93,7 @@ impl PrimitiveType { #[new] #[pyo3(signature = (data_type))] fn new(data_type: String) -> PyResult { - let data_type: DeltaPrimitve = + let data_type: DeltaPrimitive = serde_json::from_str(&format!("\"{data_type}\"")).map_err(|_| { if data_type.starts_with("decimal") { PyValueError::new_err(format!( @@ -136,22 +144,33 @@ impl PrimitiveType { } #[pyo3(text_signature = "($self)")] - fn to_pyarrow(&self) -> PyResult> { + fn to_arrow(&self) -> PyResult { let inner_type = DataType::Primitive(self.inner_type.clone()); - Ok(PyArrowType((&inner_type).try_into().map_err( - |err: ArrowError| PyException::new_err(err.to_string()), - )?)) + let arrow_type: ArrowDataType = (&inner_type) + .try_into_arrow() + .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; + + Ok(arrow_type.into()) } #[pyo3(text_signature = "(data_type)")] #[staticmethod] - fn from_pyarrow(data_type: PyArrowType) -> PyResult { - let inner_type: DataType = (&data_type.0) - .try_into() + fn from_arrow(data_type: PyDataType) -> PyResult { + let inner_type: DataType = (&data_type.into_inner()) + .try_into_kernel() .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; inner_type.try_into() } + + fn __arrow_c_schema__<'py>(&self, py: Python<'py>) -> PyArrowResult> { + let inner_type = DataType::Primitive(self.inner_type.clone()); + let arrow_type: ArrowDataType = (&inner_type) + .try_into_arrow() + .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; + + to_schema_pycapsule(py, arrow_type) + } } #[pyclass(module = "deltalake._internal")] @@ -251,23 +270,33 @@ impl ArrayType { } #[pyo3(text_signature = "($self)")] - fn to_pyarrow(&self) -> PyResult> { - Ok(PyArrowType( - (&DataType::Array(Box::new(self.inner_type.clone()))) - .try_into() - .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?, - )) + fn to_arrow(&self) -> PyResult { + let inner_type = DataType::Array(Box::new(self.inner_type.clone())); + let arrow_type: ArrowDataType = (&inner_type) + .try_into_arrow() + .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; + + Ok(arrow_type.into()) } - #[staticmethod] #[pyo3(text_signature = "(data_type)")] - fn from_pyarrow(data_type: PyArrowType) -> PyResult { - let inner_type: DataType = (&data_type.0) - .try_into() + #[staticmethod] + fn from_arrow(data_type: PyDataType) -> PyResult { + let inner_type: DataType = (&data_type.into_inner()) + .try_into_kernel() .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; inner_type.try_into() } + + fn __arrow_c_schema__<'py>(&self, py: Python<'py>) -> PyArrowResult> { + let inner_type = DataType::Array(Box::new(self.inner_type.clone())); + let arrow_type: ArrowDataType = (&inner_type) + .try_into_arrow() + .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; + + to_schema_pycapsule(py, arrow_type) + } } #[pyclass(module = "deltalake._internal")] @@ -383,23 +412,33 @@ impl MapType { } #[pyo3(text_signature = "($self)")] - fn to_pyarrow(&self) -> PyResult> { - Ok(PyArrowType( - (&DataType::Map(Box::new(self.inner_type.clone()))) - .try_into() - .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?, - )) + fn to_arrow(&self) -> PyResult { + let inner_type = DataType::Map(Box::new(self.inner_type.clone())); + let arrow_type: ArrowDataType = (&inner_type) + .try_into_arrow() + .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; + + Ok(arrow_type.into()) } #[staticmethod] #[pyo3(text_signature = "(data_type)")] - fn from_pyarrow(data_type: PyArrowType) -> PyResult { - let inner_type: DataType = (&data_type.0) - .try_into() + fn from_arrow(data_type: PyDataType) -> PyResult { + let inner_type: DataType = (&data_type.into_inner()) + .try_into_kernel() .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; inner_type.try_into() } + + fn __arrow_c_schema__<'py>(&self, py: Python<'py>) -> PyArrowResult> { + let inner_type = DataType::Map(Box::new(self.inner_type.clone())); + let arrow_type: ArrowDataType = (&inner_type) + .try_into_arrow() + .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; + + to_schema_pycapsule(py, arrow_type) + } } #[pyclass(module = "deltalake._internal")] @@ -441,12 +480,7 @@ impl Field { match v { serde_json::Value::Number(n) => n.as_i64().map_or_else( || MetadataValue::String(v.to_string()), - |i| { - i32::try_from(i) - .ok() - .map(MetadataValue::Number) - .unwrap_or_else(|| MetadataValue::String(v.to_string())) - }, + MetadataValue::Number, ), serde_json::Value::String(s) => MetadataValue::String(s.to_string()), other => MetadataValue::String(other.to_string()), @@ -532,26 +566,40 @@ impl Field { } #[pyo3(text_signature = "($self)")] - fn to_pyarrow(&self) -> PyResult> { - Ok(PyArrowType((&self.inner).try_into().map_err( - |err: ArrowError| PyException::new_err(err.to_string()), - )?)) + fn to_arrow(&self) -> PyResult { + let inner_type = self.inner.clone(); + let field: ArrowField = (&inner_type) + .try_into_arrow() + .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; + + Ok(Arc::new(field).into()) } #[staticmethod] #[pyo3(text_signature = "(field)")] - fn from_pyarrow(field: PyArrowType) -> PyResult { + fn from_arrow(field: PyField) -> PyResult { + let field = field.into_inner().as_ref().clone(); + Ok(Self { - inner: StructField::try_from(&field.0) + inner: (&field) + .try_into_kernel() .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?, }) } + + fn __arrow_c_schema__<'py>(&self, py: Python<'py>) -> PyArrowResult> { + let inner_type = self.inner.clone(); + let field: ArrowField = (&inner_type) + .try_into_arrow() + .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; + to_schema_pycapsule(py, field) + } } #[pyclass(subclass, module = "deltalake._internal")] #[derive(Clone)] pub struct StructType { - inner_type: DeltaStructType, + pub(crate) inner_type: DeltaStructType, } impl From for StructType { @@ -644,23 +692,32 @@ impl StructType { } #[pyo3(text_signature = "($self)")] - fn to_pyarrow(&self) -> PyResult> { - Ok(PyArrowType( - (&DataType::Struct(Box::new(self.inner_type.clone()))) - .try_into() - .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?, - )) + fn to_arrow(&self) -> PyResult { + let inner_type = DataType::Struct(Box::new(self.inner_type.clone())); + let arrow_type: ArrowDataType = (&inner_type) + .try_into_arrow() + .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; + + Ok(arrow_type.into()) } #[staticmethod] #[pyo3(text_signature = "(data_type)")] - fn from_pyarrow(data_type: PyArrowType) -> PyResult { - let inner_type: DataType = (&data_type.0) - .try_into() + fn from_arrow(data_type: PyDataType) -> PyResult { + let inner_type: DataType = (&data_type.into_inner()) + .try_into_kernel() .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; inner_type.try_into() } + + fn __arrow_c_schema__<'py>(&self, py: Python<'py>) -> PyArrowResult> { + let inner_type = DataType::Struct(Box::new(self.inner_type.clone())); + let arrow_type: ArrowDataType = (&inner_type) + .try_into_arrow() + .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; + to_schema_pycapsule(py, arrow_type) + } } pub fn schema_to_pyobject(schema: DeltaStructType, py: Python<'_>) -> PyResult> { @@ -682,8 +739,8 @@ pub fn schema_to_pyobject(schema: DeltaStructType, py: Python<'_>) -> PyResult>> import pyarrow as pa -/// >>> Schema.from_pyarrow(pa.schema({"x": pa.int32(), "y": pa.string()})) +/// >>> from arro3.core import DateType, Schema as ArrowSchema +/// >>> Schema.from_pyarrow(ArrowSchema({"x": DateType.int32(), "y": DateType.string()})) /// Schema([Field(x, PrimitiveType("integer"), nullable=True), Field(y, PrimitiveType("string"), nullable=True)]) #[pyclass(extends = StructType, name = "Schema", module = "deltalake._internal")] pub struct PySchema; @@ -731,13 +788,10 @@ impl PySchema { } #[pyo3(signature = (as_large_types = false))] - fn to_pyarrow( - self_: PyRef<'_, Self>, - as_large_types: bool, - ) -> PyResult> { + fn to_arrow(self_: PyRef<'_, Self>, as_large_types: bool) -> PyResult { let super_ = self_.as_ref(); let res: ArrowSchema = (&super_.inner_type.clone()) - .try_into() + .try_into_arrow() .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; fn convert_to_large_type(field: ArrowFieldRef, dt: ArrowDataType) -> ArrowFieldRef { @@ -797,20 +851,30 @@ impl PySchema { .collect::>(), ); - Ok(PyArrowType(schema)) + Ok(schema.into()) } else { - Ok(PyArrowType(res)) + Ok(res.into()) } } + fn __arrow_c_schema__<'py>( + self_: PyRef<'_, Self>, + py: Python<'py>, + ) -> PyArrowResult> { + let super_ = self_.as_ref(); + + let res: ArrowSchema = (&super_.inner_type.clone()) + .try_into_arrow() + .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; + to_schema_pycapsule(py, res) + } + #[staticmethod] #[pyo3(text_signature = "(data_type)")] - fn from_pyarrow( - data_type: PyArrowType, - py: Python<'_>, - ) -> PyResult> { - let inner_type: DeltaStructType = (&data_type.0) - .try_into() + fn from_arrow(data_type: PyArrow3Schema, py: Python<'_>) -> PyResult> { + let schema = data_type.into_inner().as_ref().clone(); + let inner_type: DeltaStructType = (&schema) + .try_into_kernel() .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; schema_to_pyobject(inner_type, py) diff --git a/python/src/utils.rs b/python/src/utils.rs index b02437cee6..fc26f22666 100644 --- a/python/src/utils.rs +++ b/python/src/utils.rs @@ -1,6 +1,8 @@ use std::sync::{Arc, OnceLock}; -use deltalake::storage::{ListResult, ObjectStore, ObjectStoreError, ObjectStoreResult, Path}; +use deltalake::logstore::object_store::{ + path::Path, Error as ObjectStoreError, ListResult, ObjectStore, Result as ObjectStoreResult, +}; use futures::future::{join_all, BoxFuture, FutureExt}; use futures::StreamExt; use pyo3::types::{IntoPyDict, PyAnyMethods, PyModule}; @@ -18,7 +20,7 @@ pub fn rt() -> &'static Runtime { "Forked process detected - current PID is {pid} but the tokio runtime was created by {runtime_pid}. The tokio \ runtime does not support forked processes https://github.com/tokio-rs/tokio/issues/4301. If you are \ seeing this message while using Python multithreading make sure to use the `spawn` or `forkserver` \ - mode.", + mode.", ); } TOKIO_RT.get_or_init(|| Runtime::new().expect("Failed to create a tokio runtime.")) diff --git a/python/src/writer.rs b/python/src/writer.rs index 0e8c20cf8f..ac32064b94 100644 --- a/python/src/writer.rs +++ b/python/src/writer.rs @@ -1,31 +1,48 @@ //! This module contains helper functions to create a LazyTableProvider from an ArrowArrayStreamReader +use std::fmt::{self}; +use std::sync::{Arc, Mutex}; -use crate::DeltaResult; -use deltalake::arrow::ffi_stream::ArrowArrayStreamReader; +use arrow_schema::SchemaRef; +use deltalake::arrow::array::RecordBatchReader; +use deltalake::arrow::error::ArrowError; +use deltalake::arrow::error::Result as ArrowResult; +use deltalake::arrow::record_batch::RecordBatch; use deltalake::datafusion::catalog::TableProvider; use deltalake::datafusion::physical_plan::memory::LazyBatchGenerator; -use deltalake::delta_datafusion::LazyTableProvider; +use deltalake::operations::cast::cast_record_batch; use parking_lot::RwLock; -use std::fmt::{self}; -use std::sync::{Arc, Mutex}; + +use crate::datafusion::LazyTableProvider; +use crate::DeltaResult; /// Convert an [ArrowArrayStreamReader] into a [LazyTableProvider] -pub fn to_lazy_table(source: ArrowArrayStreamReader) -> DeltaResult> { - use deltalake::arrow::array::RecordBatchReader; +pub fn to_lazy_table( + source: Box, +) -> DeltaResult> { let schema = source.schema(); - let arrow_stream: Arc> = Arc::new(Mutex::new(source)); let arrow_stream_batch_generator: Arc> = - Arc::new(RwLock::new(ArrowStreamBatchGenerator::new(arrow_stream))); + Arc::new(RwLock::new(ArrowStreamBatchGenerator::new(source))); Ok(Arc::new(LazyTableProvider::try_new( schema.clone(), vec![arrow_stream_batch_generator], )?)) } +pub struct ReaderWrapper { + reader: Mutex>, +} + +impl fmt::Debug for ReaderWrapper { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("ReaderWrapper") + .field("reader", &"") + .finish() + } +} #[derive(Debug)] -pub(crate) struct ArrowStreamBatchGenerator { - pub array_stream: Arc>, +pub struct ArrowStreamBatchGenerator { + pub array_stream: ReaderWrapper, } impl fmt::Display for ArrowStreamBatchGenerator { @@ -39,8 +56,12 @@ impl fmt::Display for ArrowStreamBatchGenerator { } impl ArrowStreamBatchGenerator { - pub fn new(array_stream: Arc>) -> Self { - Self { array_stream } + pub fn new(array_stream: Box) -> Self { + Self { + array_stream: ReaderWrapper { + reader: Mutex::new(array_stream), + }, + } } } @@ -48,7 +69,7 @@ impl LazyBatchGenerator for ArrowStreamBatchGenerator { fn generate_next_batch( &mut self, ) -> deltalake::datafusion::error::Result> { - let mut stream_reader = self.array_stream.lock().map_err(|_| { + let mut stream_reader = self.array_stream.reader.lock().map_err(|_| { deltalake::datafusion::error::DataFusionError::Execution( "Failed to lock the ArrowArrayStreamReader".to_string(), ) @@ -63,3 +84,45 @@ impl LazyBatchGenerator for ArrowStreamBatchGenerator { } } } + +/// A lazy casting wrapper around a RecordBatchReader +struct LazyCastReader { + input: Box, + target_schema: SchemaRef, +} + +impl RecordBatchReader for LazyCastReader { + fn schema(&self) -> SchemaRef { + self.target_schema.clone() + } +} + +impl Iterator for LazyCastReader { + type Item = ArrowResult; + + fn next(&mut self) -> Option { + match self.input.next() { + Some(Ok(batch)) => Some( + cast_record_batch(&batch, self.target_schema.clone(), false, false) + .map_err(|e| ArrowError::CastError(e.to_string())), + ), + Some(Err(e)) => Some(Err(e)), + None => None, + } + } +} + +/// Returns a boxed reader that lazily casts each batch to the provided schema. +pub fn maybe_lazy_cast_reader( + input: Box, + target_schema: SchemaRef, +) -> Box { + if !input.schema().eq(&target_schema) { + Box::new(LazyCastReader { + input, + target_schema, + }) + } else { + input + } +} diff --git a/python/tests/conftest.py b/python/tests/conftest.py index 8f85f4ab04..6828b83ba3 100644 --- a/python/tests/conftest.py +++ b/python/tests/conftest.py @@ -5,13 +5,17 @@ from datetime import date, datetime, timedelta, timezone from decimal import Decimal from time import sleep +from typing import TYPE_CHECKING -import pyarrow as pa import pytest +from arro3.core import Array, DataType, Field, Schema, Table from azure.storage import blob from deltalake import DeltaTable, WriterProperties, write_deltalake +if TYPE_CHECKING: + import pyarrow as pa + def wait_till_host_is_available(host: str, timeout_sec: int = 0.5): spacing = 2 @@ -180,8 +184,10 @@ def azurite_sas_creds(azurite_creds): @pytest.fixture() -def sample_data(): +def sample_data_pyarrow() -> "pa.Table": nrows = 5 + import pyarrow as pa + return pa.table( { "utf8": pa.array([str(x) for x in range(nrows)]), @@ -201,7 +207,10 @@ def sample_data(): [datetime(2022, 1, 1) + timedelta(hours=x) for x in range(nrows)] ), "struct": pa.array([{"x": x, "y": str(x)} for x in range(nrows)]), - "list": pa.array([list(range(x + 1)) for x in range(nrows)]), + "list": pa.array( + [list(range(x + 1)) for x in range(nrows)], + type=pa.list_(pa.field("inner", pa.int64(), nullable=False)), + ), # NOTE: https://github.com/apache/arrow-rs/issues/477 #'map': pa.array([[(str(y), y) for y in range(x)] for x in range(nrows)], pa.map_(pa.string(), pa.int64())), } @@ -209,42 +218,62 @@ def sample_data(): @pytest.fixture() -def existing_table(tmp_path: pathlib.Path, sample_data: pa.Table): +def existing_table(tmp_path: pathlib.Path, sample_data_pyarrow: "pa.Table"): path = str(tmp_path) - write_deltalake(path, sample_data) + write_deltalake(path, sample_data_pyarrow) return DeltaTable(path) @pytest.fixture() -def sample_table(): +def sample_table() -> Table: nrows = 5 - return pa.table( + return Table( { - "id": pa.array(["1", "2", "3", "4", "5"]), - "price": pa.array(list(range(nrows)), pa.int64()), - "sold": pa.array(list(range(nrows)), pa.int32()), - "deleted": pa.array([False] * nrows), - } + "id": Array( + ["1", "2", "3", "4", "5"], + Field("id", type=DataType.string(), nullable=True), + ), + "price": Array( + list(range(nrows)), Field("price", type=DataType.int64(), nullable=True) + ), + "sold": Array( + list(range(nrows)), Field("sold", type=DataType.int32(), nullable=True) + ), + "deleted": Array( + [False] * nrows, Field("deleted", type=DataType.bool(), nullable=True) + ), + }, ) @pytest.fixture() -def existing_sample_table(tmp_path: pathlib.Path, sample_table: pa.Table): +def existing_sample_table(tmp_path: pathlib.Path, sample_table: Table): path = str(tmp_path) write_deltalake(path, sample_table) return DeltaTable(path) @pytest.fixture() -def sample_table_with_spaces_numbers(): +def sample_table_with_spaces_numbers() -> Table: nrows = 5 - return pa.table( + return Table.from_pydict( { - "1id": pa.array(["1", "2", "3", "4", "5"]), - "price": pa.array(list(range(nrows)), pa.int64()), - "sold items": pa.array(list(range(nrows)), pa.int32()), - "deleted": pa.array([False] * nrows), - } + "1id": Array(["1", "2", "3", "4", "5"], DataType.string()), + "price": Array(list(range(nrows)), DataType.int64()), + "sold items": Array(list(range(nrows)), DataType.int32()), + "deleted": Array( + [False] * nrows, + DataType.bool(), + ), + }, + schema=Schema( + fields=[ + Field("1id", type=DataType.string(), nullable=True), + Field("price", type=DataType.int64(), nullable=True), + Field("sold items", type=DataType.int32(), nullable=True), + Field("deleted", type=DataType.bool(), nullable=True), + ] + ), ) diff --git a/python/tests/data_acceptance/test_reader.py b/python/tests/data_acceptance/test_reader.py index 7edf1e58a2..048049f4e2 100644 --- a/python/tests/data_acceptance/test_reader.py +++ b/python/tests/data_acceptance/test_reader.py @@ -3,14 +3,15 @@ import json import os from pathlib import Path -from typing import Any, NamedTuple +from typing import TYPE_CHECKING, Any, NamedTuple -import pyarrow as pa -import pyarrow.parquet as pq import pytest from deltalake import DeltaTable +if TYPE_CHECKING: + import pyarrow as pa + class ReadCase(NamedTuple): root: Path @@ -52,10 +53,13 @@ class ReadCase(NamedTuple): } +@pytest.mark.pyarrow @pytest.mark.parametrize( "case", cases, ids=lambda case: f"{case.case_info['name']} (version={case.version})" ) def test_dat(case: ReadCase): + import pyarrow.parquet as pq + root, version, case_info, version_metadata = case if case_info["name"] in failing_cases: @@ -89,7 +93,9 @@ def test_dat(case: ReadCase): dt.to_pyarrow_table() -def assert_tables_equal(first: pa.Table, second: pa.Table) -> None: +def assert_tables_equal(first: "pa.Table", second: "pa.Table") -> None: + import pyarrow as pa + assert first.schema == second.schema sort_keys = [ (col, "ascending") diff --git a/python/tests/pyspark_integration/test_write_to_pyspark.py b/python/tests/pyspark_integration/test_write_to_pyspark.py index 698c301239..97353e45b3 100644 --- a/python/tests/pyspark_integration/test_write_to_pyspark.py +++ b/python/tests/pyspark_integration/test_write_to_pyspark.py @@ -3,29 +3,24 @@ import os import pathlib -import pyarrow as pa import pytest +from arro3.core import Array, DataType, Table +from arro3.core import Field as ArrowField from deltalake import DeltaTable, write_deltalake from deltalake.exceptions import DeltaProtocolError from .utils import assert_spark_read_equal, get_spark, run_stream_with_checkpoint -try: - import delta - import delta.pip_utils - import delta.tables - import pyspark - - spark = get_spark() -except ModuleNotFoundError: - pass - @pytest.mark.pyspark +@pytest.mark.pyarrow @pytest.mark.integration def test_write_basic(tmp_path: pathlib.Path): # Write table in Spark + import pyarrow as pa + import pyspark + spark = get_spark() schema = pyspark.sql.types.StructType( [ @@ -43,6 +38,7 @@ def test_write_basic(tmp_path: pathlib.Path): ) # Overwrite table in deltalake data = pa.table({"c1": pa.array([5, 6], type=pa.int32())}) + write_deltalake(str(tmp_path), data, mode="overwrite") # Read table in Spark @@ -50,10 +46,17 @@ def test_write_basic(tmp_path: pathlib.Path): @pytest.mark.pyspark +@pytest.mark.pyarrow @pytest.mark.integration def test_write_invariant(tmp_path: pathlib.Path): # Write table in Spark with invariant + # spark = get_spark() + import delta + import delta.pip_utils + import delta.tables + import pyarrow as pa + import pyspark schema = pyspark.sql.types.StructType( [ @@ -79,7 +82,15 @@ def test_write_invariant(tmp_path: pathlib.Path): ) # Cannot write invalid data to the table - invalid_data = pa.table({"c1": pa.array([6, 2], type=pa.int32())}) + invalid_data = Table( + { + "c1": Array( + [6, 2], + ArrowField("c1", type=DataType.int32(), nullable=True), + ), + } + ) + with pytest.raises( DeltaProtocolError, match=r"Invariant \(c1 > 3\) violated by value .+2" ): @@ -95,8 +106,11 @@ def test_write_invariant(tmp_path: pathlib.Path): @pytest.mark.pyspark +@pytest.mark.pyarrow @pytest.mark.integration def test_spark_read_optimize_history(tmp_path: pathlib.Path): + import pyarrow as pa + ids = ["1"] * 10 values = [10, 20, 30, 40, 50, 60, 70, 80, 90, 100] @@ -123,8 +137,11 @@ def test_spark_read_optimize_history(tmp_path: pathlib.Path): @pytest.mark.pyspark +@pytest.mark.pyarrow @pytest.mark.integration def test_spark_read_z_ordered_history(tmp_path: pathlib.Path): + import pyarrow as pa + ids = ["1"] * 10 values = [10, 20, 30, 40, 50, 60, 70, 80, 90, 100] @@ -151,8 +168,11 @@ def test_spark_read_z_ordered_history(tmp_path: pathlib.Path): @pytest.mark.pyspark +@pytest.mark.pyarrow @pytest.mark.integration def test_spark_read_repair_run(tmp_path): + import pyarrow as pa + ids = ["1"] * 10 values = [10, 20, 30, 40, 50, 60, 70, 80, 90, 100] @@ -179,6 +199,7 @@ def test_spark_read_repair_run(tmp_path): @pytest.mark.pyspark +@pytest.mark.pyarrow @pytest.mark.integration def test_spark_stream_schema_evolution(tmp_path: pathlib.Path): """https://github.com/delta-io/delta-rs/issues/3274""" @@ -187,6 +208,7 @@ def test_spark_stream_schema_evolution(tmp_path: pathlib.Path): a schema evolution write, since old behavior was to generate a new table ID between schema evolution runs, which caused Spark to error thinking the table had changed. """ + import pyarrow as pa data_first_write = pa.array( [ diff --git a/python/tests/pyspark_integration/test_writer_readable.py b/python/tests/pyspark_integration/test_writer_readable.py index 6d1a528ee6..d7254b90ee 100644 --- a/python/tests/pyspark_integration/test_writer_readable.py +++ b/python/tests/pyspark_integration/test_writer_readable.py @@ -1,14 +1,17 @@ """Test that pyspark can read tables written by deltalake(delta-rs).""" import pathlib +from typing import TYPE_CHECKING -import pyarrow as pa import pytest from deltalake import DeltaTable, write_deltalake from .utils import assert_spark_read_equal, get_spark +if TYPE_CHECKING: + import pyarrow as pa + try: import delta import delta.pip_utils @@ -22,10 +25,10 @@ @pytest.mark.pyspark @pytest.mark.integration -def test_basic_read(sample_data: pa.Table, existing_table: DeltaTable): +def test_basic_read(sample_data_pyarrow: "pa.Table", existing_table: DeltaTable): uri = existing_table._table.table_uri() + "/" - assert_spark_read_equal(sample_data, uri) + assert_spark_read_equal(sample_data_pyarrow, uri) dt = delta.tables.DeltaTable.forPath(spark, uri) history = dt.history().collect() @@ -34,18 +37,20 @@ def test_basic_read(sample_data: pa.Table, existing_table: DeltaTable): @pytest.mark.pyspark +@pytest.mark.pyarrow @pytest.mark.integration -def test_partitioned(tmp_path: pathlib.Path, sample_data: pa.Table): +def test_partitioned(tmp_path: pathlib.Path, sample_data_pyarrow: "pa.Table"): partition_cols = ["date32", "utf8", "timestamp", "bool"] + import pyarrow as pa # Add null values to sample data to verify we can read null partitions - sample_data_with_null = sample_data + sample_data_with_null = sample_data_pyarrow for col in partition_cols: - i = sample_data.schema.get_field_index(col) - field = sample_data.schema.field(i) - nulls = pa.array([None] * sample_data.num_rows, type=field.type) + i = sample_data_pyarrow.schema.get_field_index(col) + field = sample_data_pyarrow.schema.field(i) + nulls = pa.array([None] * sample_data_pyarrow.num_rows, type=field.type) sample_data_with_null = sample_data_with_null.set_column(i, field, nulls) - data = pa.concat_tables([sample_data, sample_data_with_null]) + data = pa.concat_tables([sample_data_pyarrow, sample_data_with_null]) write_deltalake(str(tmp_path), data, partition_by=partition_cols) @@ -53,23 +58,29 @@ def test_partitioned(tmp_path: pathlib.Path, sample_data: pa.Table): @pytest.mark.pyspark +@pytest.mark.pyarrow @pytest.mark.integration def test_overwrite( - tmp_path: pathlib.Path, sample_data: pa.Table, existing_table: DeltaTable + tmp_path: pathlib.Path, sample_data_pyarrow: "pa.Table", existing_table: DeltaTable ): + import pyarrow as pa + path = str(tmp_path) - write_deltalake(path, sample_data, mode="append") - expected = pa.concat_tables([sample_data, sample_data]) + write_deltalake(path, sample_data_pyarrow, mode="append") + expected = pa.concat_tables([sample_data_pyarrow, sample_data_pyarrow]) assert_spark_read_equal(expected, path) - write_deltalake(path, sample_data, mode="overwrite") - assert_spark_read_equal(sample_data, path) + write_deltalake(path, sample_data_pyarrow, mode="overwrite") + assert_spark_read_equal(sample_data_pyarrow, path) @pytest.mark.pyspark +@pytest.mark.pyarrow @pytest.mark.integration def test_issue_1591_roundtrip_special_characters(tmp_path: pathlib.Path): + import pyarrow as pa + test_string = r'$%&/()=^"[]#*?.:_-{=}|`<>~/\r\n+' poisoned = "}|`<>~" for char in poisoned: @@ -99,8 +110,11 @@ def test_issue_1591_roundtrip_special_characters(tmp_path: pathlib.Path): @pytest.mark.pyspark +@pytest.mark.pyarrow @pytest.mark.integration def test_read_checkpointed_table(tmp_path: pathlib.Path): + import pyarrow as pa + data = pa.table( { "int": pa.array([1]), @@ -115,10 +129,13 @@ def test_read_checkpointed_table(tmp_path: pathlib.Path): @pytest.mark.pyspark +@pytest.mark.pyarrow @pytest.mark.integration def test_read_checkpointed_features_table(tmp_path: pathlib.Path): from datetime import datetime + import pyarrow as pa + data = pa.table( { "timestamp": pa.array([datetime(2010, 1, 1)]), diff --git a/python/tests/pyspark_integration/utils.py b/python/tests/pyspark_integration/utils.py index 150a4d75b6..bd38c7d684 100644 --- a/python/tests/pyspark_integration/utils.py +++ b/python/tests/pyspark_integration/utils.py @@ -1,22 +1,15 @@ -import pyarrow as pa +from typing import TYPE_CHECKING -try: +if TYPE_CHECKING: + import pyarrow as pa + + +def get_spark(): import delta import delta.pip_utils import delta.tables import pyspark -except ModuleNotFoundError: - pass - -try: - from pandas.testing import assert_frame_equal -except ModuleNotFoundError: - _has_pandas = False -else: - _has_pandas = True - -def get_spark(): builder = ( pyspark.sql.SparkSession.builder.appName("MyApp") .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") @@ -29,8 +22,10 @@ def get_spark(): def assert_spark_read_equal( - expected: pa.Table, uri: str, sort_by: list[str] = ["int32"] + expected: "pa.Table", uri: str, sort_by: list[str] = ["int32"] ): + from pandas.testing import assert_frame_equal + spark = get_spark() df = spark.read.format("delta").load(uri) diff --git a/python/tests/test_alter.py b/python/tests/test_alter.py index 56c7c99648..8dff1b2488 100644 --- a/python/tests/test_alter.py +++ b/python/tests/test_alter.py @@ -1,14 +1,19 @@ import pathlib +from typing import TYPE_CHECKING -import pyarrow as pa import pytest +from arro3.core import Array, DataType, Field, Schema, Table from deltalake import CommitProperties, DeltaTable, TableFeatures, write_deltalake from deltalake.exceptions import DeltaError, DeltaProtocolError -from deltalake.schema import Field, PrimitiveType, StructType +from deltalake.schema import Field as DeltaField +from deltalake.schema import PrimitiveType, StructType +if TYPE_CHECKING: + pass -def test_add_constraint(tmp_path: pathlib.Path, sample_table: pa.Table): + +def test_add_constraint(tmp_path: pathlib.Path, sample_table: Table): write_deltalake(tmp_path, sample_table) dt = DeltaTable(tmp_path) @@ -28,18 +33,27 @@ def test_add_constraint(tmp_path: pathlib.Path, sample_table: pa.Table): dt.alter.add_constraint({"check_price": "price < 0"}) with pytest.raises(DeltaProtocolError): - data = pa.table( + data = Table( { - "id": pa.array(["1"]), - "price": pa.array([-1], pa.int64()), - "sold": pa.array(list(range(1)), pa.int32()), - "deleted": pa.array([False] * 1), - } + "id": Array(["1"], DataType.string()), + "price": Array([-1], DataType.int64()), + "sold": Array(list(range(1)), DataType.int32()), + "deleted": Array([False] * 1, DataType.bool()), + }, + schema=Schema( + fields=[ + Field("id", type=DataType.string(), nullable=True), + Field("price", type=DataType.int64(), nullable=True), + Field("sold", type=DataType.int32(), nullable=True), + Field("deleted", type=DataType.bool(), nullable=True), + ] + ), ) + write_deltalake(tmp_path, data, mode="append") -def test_add_multiple_constraints(tmp_path: pathlib.Path, sample_table: pa.Table): +def test_add_multiple_constraints(tmp_path: pathlib.Path, sample_table: Table): write_deltalake(tmp_path, sample_table) dt = DeltaTable(tmp_path) @@ -50,9 +64,7 @@ def test_add_multiple_constraints(tmp_path: pathlib.Path, sample_table: pa.Table ) -def test_add_constraint_roundtrip_metadata( - tmp_path: pathlib.Path, sample_table: pa.Table -): +def test_add_constraint_roundtrip_metadata(tmp_path: pathlib.Path, sample_table: Table): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) @@ -65,7 +77,7 @@ def test_add_constraint_roundtrip_metadata( assert dt.history(1)[0]["userName"] == "John Doe" -def test_drop_constraint(tmp_path: pathlib.Path, sample_table: pa.Table): +def test_drop_constraint(tmp_path: pathlib.Path, sample_table: Table): write_deltalake(tmp_path, sample_table) dt = DeltaTable(tmp_path) @@ -80,7 +92,7 @@ def test_drop_constraint(tmp_path: pathlib.Path, sample_table: pa.Table): assert dt.protocol().min_writer_version == 3 -def test_drop_constraint_invalid(tmp_path: pathlib.Path, sample_table: pa.Table): +def test_drop_constraint_invalid(tmp_path: pathlib.Path, sample_table: Table): write_deltalake(tmp_path, sample_table) dt = DeltaTable(tmp_path) @@ -95,7 +107,7 @@ def test_drop_constraint_invalid(tmp_path: pathlib.Path, sample_table: pa.Table) assert dt.protocol().min_writer_version == 3 -def test_drop_constraint_invalid_ignore(tmp_path: pathlib.Path, sample_table: pa.Table): +def test_drop_constraint_invalid_ignore(tmp_path: pathlib.Path, sample_table: Table): write_deltalake(tmp_path, sample_table) dt = DeltaTable(tmp_path) @@ -105,7 +117,7 @@ def test_drop_constraint_invalid_ignore(tmp_path: pathlib.Path, sample_table: pa def test_drop_constraint_roundtrip_metadata( - tmp_path: pathlib.Path, sample_table: pa.Table + tmp_path: pathlib.Path, sample_table: Table ): write_deltalake( tmp_path, @@ -125,7 +137,7 @@ def test_drop_constraint_roundtrip_metadata( @pytest.mark.parametrize("min_writer_version", ["2", "3", "4", "5", "6", "7"]) def test_set_table_properties_min_writer_version( tmp_path: pathlib.Path, - sample_table: pa.Table, + sample_table: Table, min_writer_version: str, ): write_deltalake( @@ -146,7 +158,7 @@ def test_set_table_properties_min_writer_version( def test_set_table_properties_invalid_min_writer_version( - tmp_path: pathlib.Path, sample_table: pa.Table + tmp_path: pathlib.Path, sample_table: Table ): write_deltalake( tmp_path, @@ -166,7 +178,7 @@ def test_set_table_properties_invalid_min_writer_version( @pytest.mark.parametrize("min_reader_version", ["1", "2", "3"]) def test_set_table_properties_min_reader_version( tmp_path: pathlib.Path, - sample_table: pa.Table, + sample_table: Table, min_reader_version: str, ): write_deltalake( @@ -185,7 +197,7 @@ def test_set_table_properties_min_reader_version( def test_set_table_properties_invalid_min_reader_version( - tmp_path: pathlib.Path, sample_table: pa.Table + tmp_path: pathlib.Path, sample_table: Table ): write_deltalake( tmp_path, @@ -202,9 +214,7 @@ def test_set_table_properties_invalid_min_reader_version( assert protocol.min_writer_version == 2 -def test_set_table_properties_enable_cdf( - tmp_path: pathlib.Path, sample_table: pa.Table -): +def test_set_table_properties_enable_cdf(tmp_path: pathlib.Path, sample_table: Table): write_deltalake( tmp_path, sample_table, @@ -220,7 +230,7 @@ def test_set_table_properties_enable_cdf( def test_set_table_properties_enable_cdf_invalid( - tmp_path: pathlib.Path, sample_table: pa.Table + tmp_path: pathlib.Path, sample_table: Table ): write_deltalake( tmp_path, @@ -238,7 +248,7 @@ def test_set_table_properties_enable_cdf_invalid( def test_set_table_properties_enable_cdf_value_false( - tmp_path: pathlib.Path, sample_table: pa.Table + tmp_path: pathlib.Path, sample_table: Table ): write_deltalake( tmp_path, @@ -255,7 +265,7 @@ def test_set_table_properties_enable_cdf_value_false( def test_set_table_properties_enable_cdf_with_writer_version_bumped( - tmp_path: pathlib.Path, sample_table: pa.Table + tmp_path: pathlib.Path, sample_table: Table ): write_deltalake( tmp_path, @@ -278,7 +288,7 @@ def test_set_table_properties_enable_cdf_with_writer_version_bumped( def test_set_table_properties_enable_cdf_and_deletion_vectors( - tmp_path: pathlib.Path, sample_table: pa.Table + tmp_path: pathlib.Path, sample_table: Table ): write_deltalake( tmp_path, @@ -305,7 +315,7 @@ def test_set_table_properties_enable_cdf_and_deletion_vectors( def test_convert_checkConstraints_to_feature_after_version_upgrade( - tmp_path: pathlib.Path, sample_table: pa.Table + tmp_path: pathlib.Path, sample_table: Table ): write_deltalake(tmp_path, sample_table) @@ -337,7 +347,7 @@ def test_convert_checkConstraints_to_feature_after_version_upgrade( assert protocol.reader_features == ["deletionVectors"] -def test_set_table_properties_enable_dv(tmp_path: pathlib.Path, sample_table: pa.Table): +def test_set_table_properties_enable_dv(tmp_path: pathlib.Path, sample_table: Table): write_deltalake( tmp_path, sample_table, @@ -354,43 +364,44 @@ def test_set_table_properties_enable_dv(tmp_path: pathlib.Path, sample_table: pa assert protocol.reader_features == ["deletionVectors"] -def _sort_fields(fields: list[Field]) -> list[Field]: +def _sort_fields(fields: list[Field]) -> list[DeltaField]: return list(sorted(iter(fields), key=lambda x: (x.name, str(x.type)))) -def test_add_column_primitive(existing_table: DeltaTable): - current_fields = existing_table.schema().fields +def test_add_column_primitive(existing_sample_table: DeltaTable): + current_fields = existing_sample_table.schema().fields new_fields_to_add = [ - Field("foo", PrimitiveType("integer")), - Field("bar", PrimitiveType("float")), + DeltaField("foo", PrimitiveType("integer")), + DeltaField("bar", PrimitiveType("float")), ] - existing_table.alter.add_columns(new_fields_to_add) - new_fields = existing_table.schema().fields + existing_sample_table.alter.add_columns(new_fields_to_add) + new_fields = existing_sample_table.schema().fields assert _sort_fields(new_fields) == _sort_fields( [*current_fields, *new_fields_to_add] ) +@pytest.mark.pyarrow def test_add_field_in_struct_column(existing_table: DeltaTable): current_fields = existing_table.schema().fields new_fields_to_add = [ - Field("struct", StructType([Field("z", PrimitiveType("float"))])), + DeltaField("struct", StructType([DeltaField("z", PrimitiveType("float"))])), ] existing_table.alter.add_columns(new_fields_to_add) new_fields = existing_table.schema().fields - new_field = Field( + new_field = DeltaField( "struct", StructType( [ - Field("x", PrimitiveType("long")), - Field("y", PrimitiveType("string")), - Field("z", PrimitiveType("float")), + DeltaField("x", PrimitiveType("long")), + DeltaField("y", PrimitiveType("string")), + DeltaField("z", PrimitiveType("float")), ] ), ) @@ -399,7 +410,7 @@ def test_add_field_in_struct_column(existing_table: DeltaTable): ) -def test_add_timestamp_ntz_column(tmp_path: pathlib.Path, sample_table: pa.Table): +def test_add_timestamp_ntz_column(tmp_path: pathlib.Path, sample_table: Table): write_deltalake( tmp_path, sample_table, @@ -408,7 +419,7 @@ def test_add_timestamp_ntz_column(tmp_path: pathlib.Path, sample_table: pa.Table dt = DeltaTable(tmp_path) current_fields = dt.schema().fields - new_fields_to_add = Field("timestamp_ntz_col", PrimitiveType("timestamp_ntz")) + new_fields_to_add = DeltaField("timestamp_ntz_col", PrimitiveType("timestamp_ntz")) dt.alter.add_columns(new_fields_to_add) new_fields = dt.schema().fields @@ -445,6 +456,7 @@ def test_add_timestamp_ntz_column(tmp_path: pathlib.Path, sample_table: pa.Table all_features.append(features) +@pytest.mark.pyarrow @pytest.mark.parametrize("feature", all_features) def test_add_feature_variations(existing_table: DeltaTable, feature): """Existing table already has timestampNtz so it's already at v3,7""" @@ -476,7 +488,7 @@ def test_add_features_disallowed_protocol_increase(existing_sample_table: DeltaT ) -def test_add_feautres(existing_sample_table: DeltaTable): +def test_add_features(existing_sample_table: DeltaTable): existing_sample_table.alter.add_feature( feature=features, allow_protocol_versions_increase=True, @@ -505,7 +517,7 @@ def test_add_feautres(existing_sample_table: DeltaTable): ) # type: ignore -def test_set_column_metadata(tmp_path: pathlib.Path, sample_table: pa.Table): +def test_set_column_metadata(tmp_path: pathlib.Path, sample_table: Table): write_deltalake(tmp_path, sample_table) dt = DeltaTable(tmp_path) @@ -518,3 +530,113 @@ def test_set_column_metadata(tmp_path: pathlib.Path, sample_table: pa.Table): with pytest.raises(DeltaError): # Can't set metadata for non existing column. dt.alter.set_column_metadata("non_existing_column", {"comment": "my comment"}) + + +def test_set_table_name(tmp_path: pathlib.Path, sample_table: Table): + write_deltalake(tmp_path, sample_table) + + dt = DeltaTable(tmp_path) + + initial_metadata = dt.metadata() + assert initial_metadata.name is None + + dt.alter.set_table_name("my_awesome_table") + updated_metadata = dt.metadata() + + assert updated_metadata.name == "my_awesome_table" + + assert dt.version() == 1 + + last_action = dt.history(1)[0] + assert last_action["operation"] == "UPDATE TABLE METADATA" + + +def test_set_table_description(tmp_path: pathlib.Path, sample_table: Table): + write_deltalake(tmp_path, sample_table) + + dt = DeltaTable(tmp_path) + initial_metadata = dt.metadata() + assert initial_metadata.description is None + + dt.alter.set_table_description("A wonderful sample table for testing") + updated_metadata = dt.metadata() + + assert updated_metadata.description == "A wonderful sample table for testing" + + assert dt.version() == 1 + + last_action = dt.history(1)[0] + assert last_action["operation"] == "UPDATE TABLE METADATA" + + +def test_set_table_name_overwrite(tmp_path: pathlib.Path, sample_table: Table): + """Test overwriting an existing table name.""" + write_deltalake(tmp_path, sample_table) + + dt = DeltaTable(tmp_path) + dt.alter.set_table_name("initial_name") + dt.alter.set_table_name("new_name") + updated_metadata = dt.metadata() + + assert updated_metadata.name == "new_name" + + +def test_set_table_description_overwrite(tmp_path: pathlib.Path, sample_table: Table): + write_deltalake(tmp_path, sample_table) + + dt = DeltaTable(tmp_path) + dt.alter.set_table_description("initial description") + dt.alter.set_table_description("updated description") + + updated_metadata = dt.metadata() + + assert updated_metadata.description == "updated description" + + assert dt.version() == 2 + + +def test_set_table_name_character_limit(tmp_path: pathlib.Path, sample_table: Table): + write_deltalake(tmp_path, sample_table) + dt = DeltaTable(tmp_path) + + name_255_chars = "x" * 255 + dt.alter.set_table_name(name_255_chars) + assert dt.metadata().name == name_255_chars + + name_256_chars = "y" * 256 + with pytest.raises( + DeltaError, + match="Table metadata is invalid: name: Table name cannot be empty and cannot exceed 255 characters", + ): + dt.alter.set_table_name(name_256_chars) + + +def test_set_table_description_character_limit( + tmp_path: pathlib.Path, sample_table: Table +): + write_deltalake(tmp_path, sample_table) + dt = DeltaTable(tmp_path) + + desc_4000_chars = "x" * 4000 + dt.alter.set_table_description(desc_4000_chars) + assert dt.metadata().description == desc_4000_chars + + desc_4001_chars = "y" * 4001 + with pytest.raises( + DeltaError, + match="Table metadata is invalid: description: Table description cannot exceed 4000 characters", + ): + dt.alter.set_table_description(desc_4001_chars) + + +def test_set_table_metadata_name_none_should_raise_error( + tmp_path: pathlib.Path, sample_table: Table +): + write_deltalake(tmp_path, sample_table) + dt = DeltaTable(tmp_path) + + with pytest.raises( + DeltaError, + match="Table metadata is invalid: name: Table name cannot be empty and cannot exceed 255 characters", + ): + dt.alter.set_table_name("") diff --git a/python/tests/test_benchmark.py b/python/tests/test_benchmark.py index 34f94c3532..866aedf747 100644 --- a/python/tests/test_benchmark.py +++ b/python/tests/test_benchmark.py @@ -1,11 +1,10 @@ import os -import pyarrow as pa -import pyarrow.fs as pa_fs import pytest +from arro3.core import Array, ChunkedArray, DataType, Table from numpy.random import standard_normal -from deltalake import DeltaTable, write_deltalake +from deltalake import DeltaTable, QueryBuilder, write_deltalake # NOTE: make sure to run these in release mode with # MATURIN_EXTRA_ARGS=--release make develop @@ -14,13 +13,15 @@ @pytest.fixture() -def sample_table() -> pa.Table: +def sample_table() -> Table: max_size_bytes = 128 * 1024 * 1024 ncols = 20 nrows = max_size_bytes // 20 // 8 - tab = pa.table({f"x{i}": standard_normal(nrows) for i in range(ncols)}) + tab = Table.from_pydict({f"x{i}": standard_normal(nrows) for i in range(ncols)}) # Add index column for sorting - tab = tab.append_column("i", pa.array(range(nrows), type=pa.int64())) + tab = tab.append_column( + "i", ChunkedArray(Array(range(nrows), type=DataType.int64())) + ) return tab @@ -29,26 +30,36 @@ def test_benchmark_write(benchmark, sample_table, tmp_path): benchmark(write_deltalake, str(tmp_path), sample_table, mode="overwrite") dt = DeltaTable(str(tmp_path)) - assert dt.to_pyarrow_table().sort_by("i") == sample_table + assert ( + QueryBuilder().register("tbl", dt).execute("select * from tbl order by id") + == sample_table + ) +@pytest.mark.pyarrow @pytest.mark.benchmark(group="read") def test_benchmark_read(benchmark, sample_table, tmp_path): + import pyarrow as pa + write_deltalake(str(tmp_path), sample_table) dt = DeltaTable(str(tmp_path)) result = benchmark(dt.to_pyarrow_table) - assert result.sort_by("i") == sample_table + assert result.sort_by("i") == pa.table(sample_table) +@pytest.mark.pyarrow @pytest.mark.benchmark(group="read") def test_benchmark_read_pyarrow(benchmark, sample_table, tmp_path): + import pyarrow as pa + import pyarrow.fs as pa_fs + write_deltalake(str(tmp_path), sample_table) dt = DeltaTable(str(tmp_path)) fs = pa_fs.SubTreeFileSystem(str(tmp_path), pa_fs.LocalFileSystem()) result = benchmark(dt.to_pyarrow_table, filesystem=fs) - assert result.sort_by("i") == sample_table + assert result.sort_by("i") == pa.table(sample_table) @pytest.mark.benchmark(group="optimize") @@ -62,7 +73,9 @@ def test_benchmark_optimize(benchmark, sample_table, tmp_path, max_tasks): nrows = int(sample_table.num_rows / files_per_part) for part in parts: tab = sample_table.slice(0, nrows) - tab = tab.append_column("part", pa.array([part] * nrows)) + tab = tab.append_column( + "part", ChunkedArray(Array([part] * nrows), DataType.int64()) + ) for _ in range(files_per_part): write_deltalake(tmp_path, tab, mode="append", partition_by=["part"]) diff --git a/python/tests/test_casting.py b/python/tests/test_casting.py new file mode 100644 index 0000000000..65d6e6acef --- /dev/null +++ b/python/tests/test_casting.py @@ -0,0 +1,21 @@ +import pytest +from arro3.core import Array, DataType, Table + +from deltalake import write_deltalake +from deltalake.exceptions import DeltaError + + +def test_unsafe_cast(tmp_path): + tbl = Table.from_pydict({"foo": Array([1, 2, 3, 200], DataType.uint8())}) + + with pytest.raises( + DeltaError, + match="Cast error: Failed to convert into Arrow schema: Cast error: Failed to cast foo from Int8 to UInt8: Can't cast value 200 to type Int8", + ): + write_deltalake(tmp_path, tbl) + + +def test_safe_cast(tmp_path): + tbl = Table.from_pydict({"foo": Array([1, 2, 3, 4], DataType.uint8())}) + + write_deltalake(tmp_path, tbl) diff --git a/python/tests/test_cdf.py b/python/tests/test_cdf.py index 94f6570112..38f61bf5fa 100644 --- a/python/tests/test_cdf.py +++ b/python/tests/test_cdf.py @@ -1,28 +1,30 @@ import os from datetime import date, datetime +from typing import TYPE_CHECKING -import pyarrow as pa -import pyarrow.compute as pc -import pyarrow.dataset as ds -import pyarrow.parquet as pq import pytest +from arro3.core import Array, DataType, Field, Table from deltalake import DeltaTable, write_deltalake from deltalake.exceptions import DeltaError +if TYPE_CHECKING: + import pyarrow as pa + def test_read_cdf_partitioned_with_predicate(): dt = DeltaTable("../crates/test/tests/data/cdf-table/") - data = dt.load_cdf(0, 3, predicate="birthday = '2023-12-25'").read_all().to_pydict() - values = list(set(data["birthday"])) + data = dt.load_cdf(0, 3, predicate="birthday = '2023-12-25'").read_all() + + values = list(set(data["birthday"].to_pylist())) assert len(values) == 1 assert values[0] == date(2023, 12, 25) def test_read_cdf_partitioned(): dt = DeltaTable("../crates/test/tests/data/cdf-table/") - b = dt.load_cdf(0, 3).read_all().to_pydict() - assert sorted(b["id"]) == [ + b = dt.load_cdf(0, 3).read_all() + assert sorted(b["id"].to_pylist()) == [ 1, 2, 2, @@ -47,7 +49,7 @@ def test_read_cdf_partitioned(): 9, 10, ] - assert sorted(b["name"]) == [ + assert sorted(b["name"].to_pylist()) == [ "Ada", "Bob", "Bob", @@ -72,7 +74,7 @@ def test_read_cdf_partitioned(): "Kate", "Steve", ] - assert sorted(b["_change_type"]) == [ + assert sorted(b["_change_type"].to_pylist()) == [ "delete", "insert", "insert", @@ -97,7 +99,7 @@ def test_read_cdf_partitioned(): "update_preimage", "update_preimage", ] - assert sorted(b["_commit_version"]) == [ + assert sorted(b["_commit_version"].to_pylist()) == [ 0, 0, 0, @@ -122,7 +124,7 @@ def test_read_cdf_partitioned(): 2, 3, ] - assert sorted(b["_commit_timestamp"]) == [ + assert sorted(b["_commit_timestamp"].to_pylist()) == [ datetime(2023, 12, 22, 17, 10, 18, 828000), datetime(2023, 12, 22, 17, 10, 18, 828000), datetime(2023, 12, 22, 17, 10, 18, 828000), @@ -147,7 +149,7 @@ def test_read_cdf_partitioned(): datetime(2023, 12, 29, 21, 41, 33, 785000), datetime(2024, 1, 6, 16, 44, 59, 570000), ] - assert sorted(b["birthday"]) == [ + assert sorted(b["birthday"].to_pylist()) == [ date(2023, 12, 22), date(2023, 12, 22), date(2023, 12, 22), @@ -176,9 +178,9 @@ def test_read_cdf_partitioned(): def test_read_cdf_non_partitioned(): dt = DeltaTable("../crates/test/tests/data/cdf-table-non-partitioned/") - b = dt.load_cdf(0, 3).read_all().to_pydict() + b = dt.load_cdf(0, 3).read_all() - assert sorted(b["id"]) == [ + assert sorted(b["id"].to_pylist()) == [ 1, 2, 2, @@ -203,7 +205,7 @@ def test_read_cdf_non_partitioned(): 9, 10, ] - assert sorted(b["name"]) == [ + assert sorted(b["name"].to_pylist()) == [ "Ada", "Bob", "Bob", @@ -228,7 +230,7 @@ def test_read_cdf_non_partitioned(): "Kate", "Steve", ] - assert sorted(b["birthday"]) == [ + assert sorted(b["birthday"].to_pylist()) == [ date(2024, 4, 14), date(2024, 4, 14), date(2024, 4, 14), @@ -253,7 +255,7 @@ def test_read_cdf_non_partitioned(): date(2024, 4, 17), date(2024, 4, 17), ] - assert sorted(b["long_field"]) == [ + assert sorted(b["long_field"].to_pylist()) == [ 1, 1, 1, @@ -278,7 +280,7 @@ def test_read_cdf_non_partitioned(): 8, 99999999999999999, ] - assert sorted(b["boolean_field"]) == [ + assert sorted(b["boolean_field"].to_pylist()) == [ True, True, True, @@ -303,7 +305,7 @@ def test_read_cdf_non_partitioned(): True, True, ] - assert sorted(b["double_field"]) == [ + assert sorted(b["double_field"].to_pylist()) == [ 3.14, 3.14, 3.14, @@ -328,7 +330,7 @@ def test_read_cdf_non_partitioned(): 3.14, 3.14, ] - assert sorted(b["smallint_field"]) == [ + assert sorted(b["smallint_field"].to_pylist()) == [ 1, 1, 1, @@ -353,7 +355,7 @@ def test_read_cdf_non_partitioned(): 1, 1, ] - assert sorted(b["_change_type"]) == [ + assert sorted(b["_change_type"].to_pylist()) == [ "delete", "insert", "insert", @@ -378,7 +380,7 @@ def test_read_cdf_non_partitioned(): "update_preimage", "update_preimage", ] - assert sorted(b["_commit_version"]) == [ + assert sorted(b["_commit_version"].to_pylist()) == [ 0, 0, 0, @@ -403,7 +405,7 @@ def test_read_cdf_non_partitioned(): 2, 3, ] - assert sorted(b["_commit_timestamp"]) == [ + assert sorted(b["_commit_timestamp"].to_pylist()) == [ datetime(2024, 4, 14, 15, 58, 26, 249000), datetime(2024, 4, 14, 15, 58, 26, 249000), datetime(2024, 4, 14, 15, 58, 26, 249000), @@ -436,12 +438,18 @@ def test_read_cdf_partitioned_projection(): assert columns == dt.load_cdf(0, 3, columns=columns).schema.names -def test_delete_unpartitioned_cdf(tmp_path, sample_data: pa.Table): +@pytest.mark.pyarrow +def test_delete_unpartitioned_cdf(tmp_path, sample_data_pyarrow: "pa.Table"): + import pyarrow as pa + import pyarrow.compute as pc + import pyarrow.dataset as ds + import pyarrow.parquet as pq + cdc_path = f"{tmp_path}/_change_data" write_deltalake( tmp_path, - sample_data, + sample_data_pyarrow, mode="append", configuration={"delta.enableChangeDataFeed": "true"}, ) @@ -449,7 +457,7 @@ def test_delete_unpartitioned_cdf(tmp_path, sample_data: pa.Table): dt.delete("int64 > 2") expected_data = ( - ds.dataset(sample_data) + ds.dataset(sample_data_pyarrow) .to_table(filter=(pc.field("int64") > 2)) .append_column( field_=pa.field("_change_type", pa.string(), nullable=False), @@ -462,12 +470,18 @@ def test_delete_unpartitioned_cdf(tmp_path, sample_data: pa.Table): assert cdc_data == expected_data -def test_delete_partitioned_cdf(tmp_path, sample_data: pa.Table): +@pytest.mark.pyarrow +def test_delete_partitioned_cdf(tmp_path, sample_data_pyarrow: "pa.Table"): + import pyarrow as pa + import pyarrow.compute as pc + import pyarrow.dataset as ds + import pyarrow.parquet as pq + cdc_path = f"{tmp_path}/_change_data" write_deltalake( tmp_path, - sample_data, + sample_data_pyarrow, mode="overwrite", partition_by=["utf8"], configuration={"delta.enableChangeDataFeed": "true"}, @@ -476,14 +490,15 @@ def test_delete_partitioned_cdf(tmp_path, sample_data: pa.Table): dt.delete("int64 > 2") expected_data = ( - ds.dataset(sample_data) + ds.dataset(sample_data_pyarrow) .to_table(filter=(pc.field("int64") > 2)) .append_column( field_=pa.field("_change_type", pa.string(), nullable=False), column=[["delete"] * 2], ) ) - table_schema = dt.schema().to_pyarrow() + + table_schema = pa.schema(dt.schema()) table_schema = table_schema.insert( len(table_schema), pa.field("_change_type", pa.string(), nullable=False) ) @@ -494,12 +509,18 @@ def test_delete_partitioned_cdf(tmp_path, sample_data: pa.Table): assert cdc_data == expected_data -def test_write_predicate_unpartitioned_cdf(tmp_path, sample_data: pa.Table): +@pytest.mark.pyarrow +def test_write_predicate_unpartitioned_cdf(tmp_path, sample_data_pyarrow: "pa.Table"): + import pyarrow as pa + import pyarrow.compute as pc + import pyarrow.dataset as ds + import pyarrow.parquet as pq + cdc_path = f"{tmp_path}/_change_data" write_deltalake( tmp_path, - sample_data, + sample_data_pyarrow, mode="append", configuration={"delta.enableChangeDataFeed": "true"}, ) @@ -507,7 +528,7 @@ def test_write_predicate_unpartitioned_cdf(tmp_path, sample_data: pa.Table): dt = DeltaTable(tmp_path) write_deltalake( dt, - data=ds.dataset(sample_data).to_table(filter=(pc.field("int64") > 2)), + data=ds.dataset(sample_data_pyarrow).to_table(filter=(pc.field("int64") > 2)), mode="overwrite", predicate="int64 > 2", configuration={"delta.enableChangeDataFeed": "true"}, @@ -515,13 +536,13 @@ def test_write_predicate_unpartitioned_cdf(tmp_path, sample_data: pa.Table): expected_data = pa.concat_tables( [ - ds.dataset(sample_data) + ds.dataset(sample_data_pyarrow) .to_table(filter=(pc.field("int64") > 2)) .append_column( field_=pa.field("_change_type", pa.string(), nullable=False), column=[["delete"] * 2], ), - ds.dataset(sample_data) + ds.dataset(sample_data_pyarrow) .to_table(filter=(pc.field("int64") > 2)) .append_column( field_=pa.field("_change_type", pa.string(), nullable=False), @@ -535,15 +556,21 @@ def test_write_predicate_unpartitioned_cdf(tmp_path, sample_data: pa.Table): assert cdc_data.sort_by([("_change_type", "ascending")]) == expected_data.sort_by( [("_change_type", "ascending")] ) - assert dt.to_pyarrow_table().sort_by([("utf8", "ascending")]) == sample_data + assert dt.to_pyarrow_table().sort_by([("utf8", "ascending")]) == sample_data_pyarrow -def test_write_predicate_partitioned_cdf(tmp_path, sample_data: pa.Table): +@pytest.mark.pyarrow +def test_write_predicate_partitioned_cdf(tmp_path, sample_data_pyarrow: "pa.Table"): + import pyarrow as pa + import pyarrow.compute as pc + import pyarrow.dataset as ds + import pyarrow.parquet as pq + cdc_path = f"{tmp_path}/_change_data" write_deltalake( tmp_path, - sample_data, + sample_data_pyarrow, mode="overwrite", partition_by=["utf8"], configuration={"delta.enableChangeDataFeed": "true"}, @@ -551,7 +578,7 @@ def test_write_predicate_partitioned_cdf(tmp_path, sample_data: pa.Table): dt = DeltaTable(tmp_path) write_deltalake( dt, - data=ds.dataset(sample_data).to_table(filter=(pc.field("int64") > 3)), + data=ds.dataset(sample_data_pyarrow).to_table(filter=(pc.field("int64") > 3)), mode="overwrite", predicate="int64 > 3", configuration={"delta.enableChangeDataFeed": "true"}, @@ -559,13 +586,13 @@ def test_write_predicate_partitioned_cdf(tmp_path, sample_data: pa.Table): expected_data = pa.concat_tables( [ - ds.dataset(sample_data) + ds.dataset(sample_data_pyarrow) .to_table(filter=(pc.field("int64") > 3)) .append_column( field_=pa.field("_change_type", pa.string(), nullable=False), column=[["delete"] * 1], ), - ds.dataset(sample_data) + ds.dataset(sample_data_pyarrow) .to_table(filter=(pc.field("int64") > 3)) .append_column( field_=pa.field("_change_type", pa.string(), nullable=False), @@ -574,7 +601,7 @@ def test_write_predicate_partitioned_cdf(tmp_path, sample_data: pa.Table): ] ) - table_schema = dt.schema().to_pyarrow() + table_schema = pa.schema(dt.schema()) table_schema = table_schema.insert( len(table_schema), pa.field("_change_type", pa.string(), nullable=False) ) @@ -588,15 +615,19 @@ def test_write_predicate_partitioned_cdf(tmp_path, sample_data: pa.Table): cdc_data = cdc_data.combine_chunks().sort_by([("_change_type", "ascending")]) assert expected_data == cdc_data - assert dt.to_pyarrow_table().sort_by([("utf8", "ascending")]) == sample_data + assert dt.to_pyarrow_table().sort_by([("utf8", "ascending")]) == sample_data_pyarrow + +@pytest.mark.pyarrow +def test_write_overwrite_unpartitioned_cdf(tmp_path, sample_data_pyarrow: "pa.Table"): + import pyarrow as pa + import pyarrow.dataset as ds -def test_write_overwrite_unpartitioned_cdf(tmp_path, sample_data: pa.Table): cdc_path = f"{tmp_path}/_change_data" write_deltalake( tmp_path, - sample_data, + sample_data_pyarrow, mode="append", configuration={"delta.enableChangeDataFeed": "true"}, ) @@ -604,13 +635,13 @@ def test_write_overwrite_unpartitioned_cdf(tmp_path, sample_data: pa.Table): dt = DeltaTable(tmp_path) write_deltalake( dt, - data=ds.dataset(sample_data).to_table(), + data=ds.dataset(sample_data_pyarrow).to_table(), mode="overwrite", configuration={"delta.enableChangeDataFeed": "true"}, ) sort_values = [("_change_type", "ascending"), ("utf8", "ascending")] expected_data = ( - ds.dataset(pa.concat_tables([sample_data] * 3)) + ds.dataset(pa.concat_tables([sample_data_pyarrow] * 3)) .to_table() .append_column( field_=pa.field("_change_type", pa.string(), nullable=True), @@ -622,30 +653,34 @@ def test_write_overwrite_unpartitioned_cdf(tmp_path, sample_data: pa.Table): "_change_data shouldn't exist since table was overwritten" ) - ## TODO(ion): check if you see insert and deletes in commit version 1 + tbl = dt.load_cdf().read_all() + + select_cols = [ + col + for col in tbl.column_names + if col not in ["_commit_version", "_commit_timestamp"] + ] + assert pa.table(tbl.select(select_cols)).sort_by(sort_values) == expected_data + assert dt.to_pyarrow_table().sort_by([("utf8", "ascending")]) == sample_data_pyarrow - assert ( - dt.load_cdf() - .read_all() - .drop_columns(["_commit_version", "_commit_timestamp"]) - .sort_by(sort_values) - == expected_data - ) - assert dt.to_pyarrow_table().sort_by([("utf8", "ascending")]) == sample_data +@pytest.mark.pyarrow +def test_write_overwrite_partitioned_cdf(tmp_path, sample_data_pyarrow: "pa.Table"): + import pyarrow as pa + import pyarrow.compute as pc + import pyarrow.dataset as ds -def test_write_overwrite_partitioned_cdf(tmp_path, sample_data: pa.Table): cdc_path = f"{tmp_path}/_change_data" write_deltalake( tmp_path, - sample_data, + sample_data_pyarrow, mode="append", partition_by=["int64"], configuration={"delta.enableChangeDataFeed": "true"}, ) - batch2 = ds.dataset(sample_data).to_table(filter=(pc.field("int64") > 3)) + batch2 = ds.dataset(sample_data_pyarrow).to_table(filter=(pc.field("int64") > 3)) dt = DeltaTable(tmp_path) write_deltalake( @@ -657,14 +692,14 @@ def test_write_overwrite_partitioned_cdf(tmp_path, sample_data: pa.Table): configuration={"delta.enableChangeDataFeed": "true"}, ) - table_schema = dt.schema().to_pyarrow() + table_schema = pa.schema(dt.schema()) table_schema = table_schema.insert( len(table_schema), pa.field("_change_type", pa.string(), nullable=False) ) sort_values = [("_change_type", "ascending"), ("utf8", "ascending")] - first_batch = sample_data.append_column( + first_batch = sample_data_pyarrow.append_column( field_=pa.field("_change_type", pa.string(), nullable=True), column=[["insert"] * 5], ) @@ -678,7 +713,7 @@ def test_write_overwrite_partitioned_cdf(tmp_path, sample_data: pa.Table): "_change_data shouldn't exist since a specific partition was overwritten" ) - assert dt.load_cdf().read_all().drop_columns( + assert pa.table(dt.load_cdf().read_all()).drop_columns( ["_commit_version", "_commit_timestamp"] ).sort_by(sort_values).select(expected_data.column_names) == pa.concat_tables( [first_batch, expected_data] @@ -689,12 +724,9 @@ def test_read_cdf_version_out_of_range(): dt = DeltaTable("../crates/test/tests/data/cdf-table/") with pytest.raises(DeltaError) as e: - dt.load_cdf(4).read_all().to_pydict() + dt.load_cdf(4).read_all() - assert ( - "invalid version. start version 4 is greater than end version 3" - in str(e).lower() - ) + assert "invalid table version: 4" in str(e).lower() def test_read_cdf_version_out_of_range_with_flag(): @@ -709,7 +741,7 @@ def test_read_timestamp_cdf_out_of_range(): start = "2033-12-22T17:10:21.675Z" with pytest.raises(DeltaError) as e: - dt.load_cdf(starting_timestamp=start).read_all().to_pydict() + dt.load_cdf(starting_timestamp=start).read_all() assert "is greater than latest commit timestamp" in str(e).lower() @@ -724,13 +756,20 @@ def test_read_timestamp_cdf_out_of_range_with_flag(): def test_read_cdf_last_version(tmp_path): - data = pa.Table.from_pydict({"foo": [1, 2, 3]}) + data = Table.from_pydict( + {"foo": Array([1, 2, 3], type=Field("foo", DataType.int32(), nullable=True))} + ) - expected = pa.Table.from_pydict( + expected = Table.from_pydict( { - "foo": [1, 2, 3], - "_change_type": ["insert", "insert", "insert"], - "_commit_version": [0, 0, 0], + "foo": Array([1, 2, 3], type=Field("foo", DataType.int32(), nullable=True)), + "_change_type": Array( + ["insert", "insert", "insert"], + type=Field("foo", DataType.string(), nullable=True), + ), + "_commit_version": Array( + [0, 0, 0], type=Field("foo", DataType.int64(), nullable=True) + ), } ) diff --git a/python/tests/test_checkpoint.py b/python/tests/test_checkpoint.py index a7c44e45fb..6abfc0e8b2 100644 --- a/python/tests/test_checkpoint.py +++ b/python/tests/test_checkpoint.py @@ -3,25 +3,30 @@ import pathlib import shutil from datetime import date, datetime, timedelta +from typing import TYPE_CHECKING -import pyarrow as pa -import pyarrow.parquet as pq import pytest +from arro3.core import Array, DataType, Table +from arro3.core import Field as ArrowField + +from deltalake import ( + DeltaTable, + PostCommitHookProperties, + QueryBuilder, + write_deltalake, +) -from deltalake import DeltaTable, PostCommitHookProperties, write_deltalake -from deltalake.exceptions import DeltaError +if TYPE_CHECKING: + import pyarrow as pa -def test_checkpoint(tmp_path: pathlib.Path, sample_data: pa.Table): +def test_checkpoint(tmp_path: pathlib.Path, sample_table: Table): tmp_table_path = tmp_path / "path" / "to" / "table" checkpoint_path = tmp_table_path / "_delta_log" / "_last_checkpoint" last_checkpoint_path = ( tmp_table_path / "_delta_log" / "00000000000000000000.checkpoint.parquet" ) - - # TODO: Include binary after fixing issue "Json error: binary type is not supported" - sample_data = sample_data.drop(["binary"]) - write_deltalake(str(tmp_table_path), sample_data) + write_deltalake(str(tmp_table_path), sample_table) assert not checkpoint_path.exists() @@ -32,46 +37,7 @@ def test_checkpoint(tmp_path: pathlib.Path, sample_data: pa.Table): assert checkpoint_path.exists() -def test_checkpoint_without_files(tmp_path: pathlib.Path, sample_data: pa.Table): - tmp_table_path = tmp_path / "path" / "to" / "table" - checkpoint_path = tmp_table_path / "_delta_log" / "_last_checkpoint" - last_checkpoint_path = ( - tmp_table_path / "_delta_log" / "00000000000000000000.checkpoint.parquet" - ) - - # TODO: Include binary after fixing issue "Json error: binary type is not supported" - sample_data = sample_data.drop(["binary"]) - write_deltalake( - str(tmp_table_path), - sample_data, - configuration={"delta.checkpointInterval": "2"}, - ) - - assert not checkpoint_path.exists() - - delta_table = DeltaTable(str(tmp_table_path), without_files=True) - with pytest.raises( - DeltaError, - match="Table has not yet been initialized with files, therefore creating a checkpoint is not possible.", - ): - delta_table.create_checkpoint() - - for i in range(3): - write_deltalake(delta_table, sample_data, mode="append") - - assert not checkpoint_path.exists() - - delta_table = DeltaTable(str(tmp_table_path), without_files=False) - delta_table.create_checkpoint() - - assert checkpoint_path.exists() - last_checkpoint_path = ( - tmp_table_path / "_delta_log" / "00000000000000000003.checkpoint.parquet" - ) - assert last_checkpoint_path.exists() - - -def setup_cleanup_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): +def setup_cleanup_metadata(tmp_path: pathlib.Path, sample_table: Table): tmp_table_path = tmp_path / "path" / "to" / "table" first_log_path = tmp_table_path / "_delta_log" / "00000000000000000000.json" first_failed_log_path = ( @@ -83,12 +49,9 @@ def setup_cleanup_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): ) third_log_path = tmp_table_path / "_delta_log" / "00000000000000000002.json" - # TODO: Include binary after fixing issue "Json error: binary type is not supported" - sample_data = sample_data.drop(["binary"]) - # Create few log files - write_deltalake(str(tmp_table_path), sample_data) - write_deltalake(str(tmp_table_path), sample_data, mode="overwrite") + write_deltalake(str(tmp_table_path), sample_table) + write_deltalake(str(tmp_table_path), sample_table, mode="overwrite") delta_table = DeltaTable(str(tmp_table_path)) delta_table.delete() @@ -116,8 +79,8 @@ def setup_cleanup_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): return delta_table -def test_cleanup_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): - delta_table = setup_cleanup_metadata(tmp_path, sample_data) +def test_cleanup_metadata(tmp_path: pathlib.Path, sample_table: Table): + delta_table = setup_cleanup_metadata(tmp_path, sample_table) delta_table.create_checkpoint() delta_table.cleanup_metadata() @@ -139,14 +102,11 @@ def test_cleanup_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): assert second_failed_log_path.exists() -def test_cleanup_metadata_log_cleanup_hook( - tmp_path: pathlib.Path, sample_data: pa.Table -): - delta_table = setup_cleanup_metadata(tmp_path, sample_data) +def test_cleanup_metadata_log_cleanup_hook(tmp_path: pathlib.Path, sample_table: Table): + delta_table = setup_cleanup_metadata(tmp_path, sample_table) delta_table.create_checkpoint() - sample_data = sample_data.drop(["binary"]) - write_deltalake(delta_table, sample_data, mode="append") + write_deltalake(delta_table, sample_table, mode="append") tmp_table_path = tmp_path / "path" / "to" / "table" first_failed_log_path = ( @@ -167,15 +127,14 @@ def test_cleanup_metadata_log_cleanup_hook( def test_cleanup_metadata_log_cleanup_hook_disabled( - tmp_path: pathlib.Path, sample_data: pa.Table + tmp_path: pathlib.Path, sample_table: Table ): - delta_table = setup_cleanup_metadata(tmp_path, sample_data) + delta_table = setup_cleanup_metadata(tmp_path, sample_table) delta_table.create_checkpoint() - sample_data = sample_data.drop(["binary"]) write_deltalake( delta_table, - sample_data, + sample_table, mode="append", post_commithook_properties=PostCommitHookProperties(cleanup_expired_logs=False), ) @@ -198,8 +157,8 @@ def test_cleanup_metadata_log_cleanup_hook_disabled( assert second_failed_log_path.exists() -def test_cleanup_metadata_no_checkpoint(tmp_path: pathlib.Path, sample_data: pa.Table): - delta_table = setup_cleanup_metadata(tmp_path, sample_data) +def test_cleanup_metadata_no_checkpoint(tmp_path: pathlib.Path, sample_table: Table): + delta_table = setup_cleanup_metadata(tmp_path, sample_table) delta_table.cleanup_metadata() tmp_table_path = tmp_path / "path" / "to" / "table" @@ -220,9 +179,12 @@ def test_cleanup_metadata_no_checkpoint(tmp_path: pathlib.Path, sample_data: pa. assert second_failed_log_path.exists() +@pytest.mark.pyarrow def test_features_maintained_after_checkpoint(tmp_path: pathlib.Path): from datetime import datetime + import pyarrow as pa + data = pa.table( { "timestamp": pa.array([datetime(2022, 1, 1)]), @@ -242,7 +204,11 @@ def test_features_maintained_after_checkpoint(tmp_path: pathlib.Path): assert current_protocol == protocol_after_checkpoint +@pytest.mark.pyarrow def test_features_null_on_below_v3_v7(tmp_path: pathlib.Path): + import pyarrow as pa + import pyarrow.parquet as pq + data = pa.table( { "int": pa.array([1]), @@ -274,6 +240,8 @@ def test_features_null_on_below_v3_v7(tmp_path: pathlib.Path): def sample_all_types(): from datetime import timezone + import pyarrow as pa + nrows = 5 return pa.table( { @@ -305,6 +273,7 @@ def sample_all_types(): ) +@pytest.mark.pyarrow @pytest.mark.parametrize( "part_col", [ @@ -313,7 +282,7 @@ def sample_all_types(): ], ) def test_checkpoint_partition_timestamp_2380( - tmp_path: pathlib.Path, sample_all_types: pa.Table, part_col: str + tmp_path: pathlib.Path, sample_all_types: "pa.Table", part_col: str ): tmp_table_path = tmp_path / "path" / "to" / "table" checkpoint_path = tmp_table_path / "_delta_log" / "_last_checkpoint" @@ -322,10 +291,10 @@ def test_checkpoint_partition_timestamp_2380( ) # TODO: Include binary after fixing issue "Json error: binary type is not supported" - sample_data = sample_all_types.drop(["binary"]) + sample_data_pyarrow = sample_all_types.drop(["binary"]) write_deltalake( str(tmp_table_path), - sample_data, + sample_data_pyarrow, partition_by=[part_col], ) @@ -340,10 +309,16 @@ def test_checkpoint_partition_timestamp_2380( def test_checkpoint_with_binary_column(tmp_path: pathlib.Path): - data = pa.table( + data = Table( { - "intColumn": pa.array([1]), - "binaryColumn": pa.array([b"a"]), + "intColumn": Array( + [1], + ArrowField("intColumn", type=DataType.int64(), nullable=True), + ), + "binaryColumn": Array( + [b"a"], + ArrowField("binaryColumn", type=DataType.binary(), nullable=True), + ), } ) @@ -359,10 +334,19 @@ def test_checkpoint_with_binary_column(tmp_path: pathlib.Path): dt = DeltaTable(tmp_path) - assert dt.to_pyarrow_table().equals(data) + assert ( + QueryBuilder() + .register("tbl", dt) + .execute("select intColumn, binaryColumn from tbl") + .read_all() + == data + ) -def test_checkpoint_post_commit_config(tmp_path: pathlib.Path, sample_data: pa.Table): +@pytest.mark.pyarrow +def test_checkpoint_post_commit_config( + tmp_path: pathlib.Path, sample_data_pyarrow: "pa.Table" +): """Checks whether checkpoints are properly written based on commit_interval""" tmp_table_path = tmp_path / "path" / "to" / "table" checkpoint_path = tmp_table_path / "_delta_log" / "_last_checkpoint" @@ -374,11 +358,11 @@ def test_checkpoint_post_commit_config(tmp_path: pathlib.Path, sample_data: pa.T ) # TODO: Include binary after fixing issue "Json error: binary type is not supported" - sample_data = sample_data.drop(["binary"]) + sample_data_pyarrow = sample_data_pyarrow.drop(["binary"]) for i in range(2): write_deltalake( str(tmp_table_path), - sample_data, + sample_data_pyarrow, mode="append", configuration={"delta.checkpointInterval": "5"}, ) @@ -390,7 +374,7 @@ def test_checkpoint_post_commit_config(tmp_path: pathlib.Path, sample_data: pa.T for i in range(10): write_deltalake( str(tmp_table_path), - sample_data, + sample_data_pyarrow, mode="append", configuration={"delta.checkpointInterval": "5"}, ) @@ -412,7 +396,7 @@ def test_checkpoint_post_commit_config(tmp_path: pathlib.Path, sample_data: pa.T def test_checkpoint_post_commit_config_multiple_operations( - tmp_path: pathlib.Path, sample_data: pa.Table + tmp_path: pathlib.Path, sample_table: Table ): """Checks whether checkpoints are properly written based on commit_interval""" tmp_table_path = tmp_path / "path" / "to" / "table" @@ -424,12 +408,10 @@ def test_checkpoint_post_commit_config_multiple_operations( tmp_table_path / "_delta_log" / "00000000000000000009.checkpoint.parquet" ) - # TODO: Include binary after fixing issue "Json error: binary type is not supported" - sample_data = sample_data.drop(["binary", "decimal"]) for i in range(4): write_deltalake( str(tmp_table_path), - sample_data, + sample_table, mode="append", configuration={"delta.checkpointInterval": "5"}, ) @@ -448,7 +430,7 @@ def test_checkpoint_post_commit_config_multiple_operations( for i in range(4): write_deltalake( str(tmp_table_path), - sample_data, + sample_table, mode="append", configuration={"delta.checkpointInterval": "5"}, ) @@ -470,9 +452,11 @@ def test_checkpoint_post_commit_config_multiple_operations( assert delta_table.version() == 9 +@pytest.mark.pyarrow def test_checkpoint_with_nullable_false(tmp_path: pathlib.Path): tmp_table_path = tmp_path / "path" / "to" / "table" checkpoint_path = tmp_table_path / "_delta_log" / "_last_checkpoint" + import pyarrow as pa pylist = [{"year": 2023, "n_party": 0}, {"year": 2024, "n_party": 1}] my_schema = pa.schema( @@ -498,6 +482,7 @@ def test_checkpoint_with_nullable_false(tmp_path: pathlib.Path): @pytest.mark.pandas +@pytest.mark.pyarrow def test_checkpoint_with_multiple_writes(tmp_path: pathlib.Path): import pandas as pd @@ -529,6 +514,7 @@ def test_checkpoint_with_multiple_writes(tmp_path: pathlib.Path): @pytest.mark.polars +@pytest.mark.xfail(reason="polars needs update") def test_refresh_snapshot_after_log_cleanup_3057(tmp_path): """https://github.com/delta-io/delta-rs/issues/3057""" import polars as pl diff --git a/python/tests/test_constraint.py b/python/tests/test_constraint.py new file mode 100644 index 0000000000..0ac5d3d623 --- /dev/null +++ b/python/tests/test_constraint.py @@ -0,0 +1,127 @@ +import pytest +from arro3.core import Array, DataType, Field, Schema, Table + +from deltalake import DeltaTable, write_deltalake +from deltalake.exceptions import DeltaError, DeltaProtocolError + + +@pytest.fixture() +def sample_table() -> Table: + nrows = 5 + return Table( + { + "id": Array( + ["1", "2", "3", "4", "5"], + Field("id", type=DataType.string(), nullable=True), + ), + "high price": Array( + list(range(nrows)), + Field("high price", type=DataType.int64(), nullable=True), + ), + }, + ) + + +def test_not_corrupting_expression(tmp_path): + data = Table.from_pydict( + { + "b": Array([1], DataType.int64()), + "color_column": Array(["red"], DataType.string()), + }, + ) + + data2 = Table.from_pydict( + { + "b": Array([1], DataType.int64()), + "color_column": Array(["blue"], DataType.string()), + }, + ) + + write_deltalake( + tmp_path, + data, + mode="overwrite", + partition_by=["color_column"], + predicate="color_column = 'red'", + ) + write_deltalake( + tmp_path, + data2, + mode="overwrite", + partition_by=["color_column"], + predicate="color_column = 'blue'", + ) + + +def test_not_corrupting_expression_columns_spaced(tmp_path): + data = Table.from_pydict( + { + "b": Array([1], DataType.int64()), + "color column": Array(["red"], DataType.string()), + }, + ) + + data2 = Table.from_pydict( + { + "b": Array([1], DataType.int64()), + "color column": Array(["blue"], DataType.string()), + }, + ) + + write_deltalake( + tmp_path, + data, + mode="overwrite", + # partition_by=["color column"], + predicate="`color column` = 'red'", + ) + write_deltalake( + tmp_path, + data2, + mode="overwrite", + # partition_by=["color column"], + predicate="`color column` = 'blue'", + ) + + +# fmt: off + +@pytest.mark.parametrize("sql_string", [ + "`high price` >= 0", + '"high price" >= 0', + "\"high price\" >= 0" +]) +def test_add_constraint(tmp_path, sample_table: Table, sql_string: str): + write_deltalake(tmp_path, sample_table) + + dt = DeltaTable(tmp_path) + + dt.alter.add_constraint({"check_price": sql_string}) + + last_action = dt.history(1)[0] + assert last_action["operation"] == "ADD CONSTRAINT" + assert dt.version() == 1 + assert dt.metadata().configuration == { + "delta.constraints.check_price": '"high price" >= 0' + } + assert dt.protocol().min_writer_version == 3 + + with pytest.raises(DeltaError): + # Invalid constraint + dt.alter.add_constraint({"check_price": '"high price" < 0'}) + + with pytest.raises(DeltaProtocolError): + data = Table( + { + "id": Array(["1"], DataType.string()), + "high price": Array([-1], DataType.int64()), + }, + schema=Schema( + fields=[ + Field("id", type=DataType.string(), nullable=True), + Field("high price", type=DataType.int64(), nullable=True), + ] + ), + ) + + write_deltalake(tmp_path, data, mode="append") diff --git a/python/tests/test_conversion.py b/python/tests/test_conversion.py new file mode 100644 index 0000000000..391a2161a7 --- /dev/null +++ b/python/tests/test_conversion.py @@ -0,0 +1,420 @@ +import pytest +from arro3.core import DataType, Field, Schema + +from deltalake import DeltaTable, write_deltalake +from deltalake.writer._conversion import _convert_arro3_schema_to_delta + + +@pytest.mark.parametrize( + "input_schema,expected_schema", + [ + # Basic types - identity + ( + Schema(fields=[Field("foo", DataType.int64())]), + Schema(fields=[Field("foo", DataType.int64())]), + ), + ( + Schema(fields=[Field("foo", DataType.int32())]), + Schema(fields=[Field("foo", DataType.int32())]), + ), + ( + Schema(fields=[Field("foo", DataType.int16())]), + Schema(fields=[Field("foo", DataType.int16())]), + ), + # Unsigned integers to signed + ( + Schema(fields=[Field("foo", DataType.uint8())]), + Schema(fields=[Field("foo", DataType.int8())]), + ), + ( + Schema(fields=[Field("foo", DataType.uint16())]), + Schema(fields=[Field("foo", DataType.int16())]), + ), + ( + Schema(fields=[Field("foo", DataType.uint32())]), + Schema(fields=[Field("foo", DataType.int32())]), + ), + ( + Schema(fields=[Field("foo", DataType.uint64())]), + Schema(fields=[Field("foo", DataType.int64())]), + ), + # Timestamps + ( + Schema(fields=[Field("foo", DataType.timestamp("s"))]), + Schema(fields=[Field("foo", DataType.timestamp("us"))]), + ), + ( + Schema( + fields=[Field("foo", DataType.timestamp("ns", tz="Europe/Amsterdam"))] + ), + Schema(fields=[Field("foo", DataType.timestamp("us", tz="UTC"))]), + ), + # Nullability variations + ( + Schema(fields=[Field("foo", DataType.uint16(), nullable=False)]), + Schema(fields=[Field("foo", DataType.int16(), nullable=False)]), + ), + ( + Schema(fields=[Field("foo", DataType.timestamp("ns"), nullable=True)]), + Schema(fields=[Field("foo", DataType.timestamp("us"), nullable=True)]), + ), + # List of unsigned ints + ( + Schema(fields=[Field("foo", DataType.list(DataType.uint32()))]), + Schema(fields=[Field("foo", DataType.list(DataType.int32()))]), + ), + ( + Schema(fields=[Field("foo", DataType.large_list(DataType.uint8()))]), + Schema(fields=[Field("foo", DataType.large_list(DataType.int8()))]), + ), + # List with nullable inner fields + ( + Schema( + fields=[ + Field( + "my_list", + DataType.list(Field("foo", DataType.uint8(), nullable=True)), + ) + ] + ), + Schema( + fields=[ + Field( + "my_list", + DataType.list(Field("foo", DataType.int8(), nullable=True)), + ) + ] + ), + ), + # List with non-nullable inner fields + ( + Schema( + fields=[ + Field( + "my_list", + DataType.list(Field("foo", DataType.uint8(), nullable=False)), + ) + ] + ), + Schema( + fields=[ + Field( + "my_list", + DataType.list(Field("foo", DataType.int8(), nullable=False)), + ) + ] + ), + ), + # Deeply nested list + ( + Schema( + fields=[ + Field( + "deep_list", + DataType.list( + Field( + "level_1", + DataType.list( + Field( + "level_2", + DataType.list( + Field( + "value", + DataType.uint16(), + nullable=True, + ) + ), + nullable=True, + ) + ), + nullable=True, + ) + ), + ) + ] + ), + Schema( + fields=[ + Field( + "deep_list", + DataType.list( + Field( + "level_1", + DataType.list( + Field( + "level_2", + DataType.list( + Field( + "value", DataType.int16(), nullable=True + ) + ), + nullable=True, + ) + ), + nullable=True, + ) + ), + ) + ] + ), + ), + # Fixed-size list + ( + Schema(fields=[Field("foo", DataType.list(DataType.uint16(), 5))]), + Schema(fields=[Field("foo", DataType.list(DataType.int16(), 5))]), + ), + # Struct with mixed fields + ( + Schema( + fields=[ + Field( + "foo", + DataType.struct( + [ + Field("a", DataType.uint64()), + Field("b", DataType.timestamp("ns")), + Field("c", DataType.uint32()), + ] + ), + ) + ] + ), + Schema( + fields=[ + Field( + "foo", + DataType.struct( + [ + Field("a", DataType.int64()), + Field("b", DataType.timestamp("us")), + Field("c", DataType.int32()), + ] + ), + ) + ] + ), + ), + # Nested struct in list + ( + Schema( + fields=[ + Field( + "foo", + DataType.list( + DataType.struct( + [ + Field("a", DataType.uint8(), nullable=False), + Field("b", DataType.timestamp("s")), + ] + ) + ), + ) + ] + ), + Schema( + fields=[ + Field( + "foo", + DataType.list( + DataType.struct( + [ + Field("a", DataType.int8(), nullable=False), + Field("b", DataType.timestamp("us")), + ] + ) + ), + ) + ] + ), + ), + # Mixed schema with multiple fields + ( + Schema( + fields=[ + Field("a", DataType.uint16()), + Field("b", DataType.timestamp("ms", tz="Europe/Berlin")), + Field( + "d", + DataType.struct( + [ + Field("x", DataType.uint32()), + Field("y", DataType.int64()), + ] + ), + ), + ] + ), + Schema( + fields=[ + Field("a", DataType.int16()), + Field("b", DataType.timestamp("us", tz="UTC")), + Field( + "d", + DataType.struct( + [Field("x", DataType.int32()), Field("y", DataType.int64())] + ), + ), + ] + ), + ), + # Field metadata preservations + ( + Schema( + fields=[ + Field( + "foo", + DataType.uint16(), + metadata={"description": "an unsigned int"}, + ) + ] + ), + Schema( + fields=[ + Field( + "foo", + DataType.int16(), + metadata={"description": "an unsigned int"}, + ) + ] + ), + ), + ( + Schema( + fields=[ + Field( + "bar", + DataType.timestamp("ns"), + nullable=True, + metadata={"origin": "sensor_7"}, + ) + ] + ), + Schema( + fields=[ + Field( + "bar", + DataType.timestamp("us"), + nullable=True, + metadata={"origin": "sensor_7"}, + ) + ] + ), + ), + ( + Schema( + fields=[ + Field( + "record", + DataType.struct( + [ + Field( + "id", + DataType.uint32(), + metadata={"index": "primary"}, + ), + Field( + "value", + DataType.timestamp("s"), + nullable=True, + metadata={"unit": "seconds"}, + ), + ] + ), + metadata={"type": "event"}, + ) + ] + ), + Schema( + fields=[ + Field( + "record", + DataType.struct( + [ + Field( + "id", + DataType.int32(), + metadata={"index": "primary"}, + ), + Field( + "value", + DataType.timestamp("us"), + nullable=True, + metadata={"unit": "seconds"}, + ), + ] + ), + metadata={"type": "event"}, + ) + ] + ), + ), + ( + Schema( + fields=[ + Field( + "sensor_readings", + DataType.list( + Field( + "reading", + DataType.uint8(), + nullable=False, + metadata={"unit": "celsius"}, + ) + ), + metadata={"shape": "1D"}, + ) + ] + ), + Schema( + fields=[ + Field( + "sensor_readings", + DataType.list( + Field( + "reading", + DataType.int8(), + nullable=False, + metadata={"unit": "celsius"}, + ) + ), + metadata={"shape": "1D"}, + ) + ] + ), + ), + ], +) +def test_schema_conversion(input_schema: Schema, expected_schema: Schema): + assert expected_schema == _convert_arro3_schema_to_delta(input_schema) + + +@pytest.mark.pandas +def test_merge_casting_table_provider(tmp_path): + import pandas as pd + + df = pd.DataFrame( + { + "a": 1, + "ts": pd.date_range( + "2021-01-01", "2021-01-02", freq="h", tz="America/Chicago" + ), + } + ) + write_deltalake(tmp_path, df, mode="overwrite") + + df2 = pd.DataFrame( + { + "a": 2, + "ts": pd.date_range( + "2021-01-01", "2021-01-03", freq="h", tz="America/Chicago" + ), + } + ) + + dt = DeltaTable(tmp_path) + dt.merge( + df2, + predicate="source.ts = target.ts", + source_alias="source", + target_alias="target", + ).when_matched_update_all().when_not_matched_insert_all().execute() diff --git a/python/tests/test_convert_to_delta.py b/python/tests/test_convert_to_delta.py index 6caff32cf0..1bb223d3ee 100644 --- a/python/tests/test_convert_to_delta.py +++ b/python/tests/test_convert_to_delta.py @@ -1,17 +1,25 @@ import pathlib +from typing import TYPE_CHECKING -import pyarrow as pa -import pyarrow.dataset as ds import pytest from deltalake import CommitProperties, convert_to_deltalake +from deltalake._internal import Field, PrimitiveType, Schema from deltalake.exceptions import DeltaError from deltalake.table import DeltaTable +if TYPE_CHECKING: + import pyarrow as pa + + +@pytest.mark.pyarrow +def test_local_convert_to_delta( + tmp_path: pathlib.Path, sample_data_pyarrow: "pa.Table" +): + import pyarrow.dataset as ds -def test_local_convert_to_delta(tmp_path: pathlib.Path, sample_data: pa.Table): ds.write_dataset( - sample_data, + sample_data_pyarrow, tmp_path, format="parquet", existing_data_behavior="overwrite_or_ignore", @@ -37,9 +45,14 @@ def test_local_convert_to_delta(tmp_path: pathlib.Path, sample_data: pa.Table): assert dt.history()[0]["userName"] == "John Doe" -def test_convert_delta_write_modes(tmp_path: pathlib.Path, sample_data: pa.Table): +@pytest.mark.pyarrow +def test_convert_delta_write_modes( + tmp_path: pathlib.Path, sample_data_pyarrow: "pa.Table" +): + import pyarrow.dataset as ds + ds.write_dataset( - sample_data, + sample_data_pyarrow, tmp_path, format="parquet", existing_data_behavior="overwrite_or_ignore", @@ -57,9 +70,14 @@ def test_convert_delta_write_modes(tmp_path: pathlib.Path, sample_data: pa.Table convert_to_deltalake(tmp_path, mode="ignore") -def test_convert_delta_with_partitioning(tmp_path: pathlib.Path, sample_data: pa.Table): +@pytest.mark.pyarrow +def test_convert_delta_with_partitioning( + tmp_path: pathlib.Path, sample_data_pyarrow: "pa.Table" +): + import pyarrow.dataset as ds + ds.write_dataset( - sample_data, + sample_data_pyarrow, tmp_path, format="parquet", existing_data_behavior="overwrite_or_ignore", @@ -67,6 +85,8 @@ def test_convert_delta_with_partitioning(tmp_path: pathlib.Path, sample_data: pa partitioning_flavor="hive", ) + schema = Schema(fields=[Field("utf8", PrimitiveType("string"))]) + with pytest.raises( DeltaError, match="Generic error: The schema of partition columns must be provided to convert a Parquet table to a Delta table", @@ -79,7 +99,7 @@ def test_convert_delta_with_partitioning(tmp_path: pathlib.Path, sample_data: pa ): convert_to_deltalake( tmp_path, - partition_by=pa.schema([pa.field("utf8", pa.string())]), + partition_by=schema, ) with pytest.raises( @@ -88,12 +108,12 @@ def test_convert_delta_with_partitioning(tmp_path: pathlib.Path, sample_data: pa ): convert_to_deltalake( tmp_path, - partition_by=pa.schema([pa.field("utf8", pa.string())]), - partition_strategy="directory", + partition_by=schema, + partition_strategy="directory", # type: ignore ) convert_to_deltalake( tmp_path, - partition_by=pa.schema([pa.field("utf8", pa.string())]), + partition_by=schema, partition_strategy="hive", ) diff --git a/python/tests/test_create.py b/python/tests/test_create.py index 3066bedac1..84910f30c6 100644 --- a/python/tests/test_create.py +++ b/python/tests/test_create.py @@ -1,16 +1,32 @@ import pathlib +from typing import TYPE_CHECKING -import pyarrow as pa import pytest +from arro3.core import Table -from deltalake import CommitProperties, DeltaTable, write_deltalake +from deltalake import CommitProperties, DeltaTable, Field, Schema, write_deltalake from deltalake.exceptions import DeltaError +from deltalake.schema import PrimitiveType +if TYPE_CHECKING: + pass -def test_create_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): +schema = Schema( + fields=[ + Field("id", type=PrimitiveType("string"), nullable=True), + Field("price", type=PrimitiveType("integer"), nullable=True), + Field("timestamp", type=PrimitiveType("timestamp_ntz"), nullable=True), + Field("deleted", type=PrimitiveType("boolean"), nullable=True), + ] +) + + +def test_create_roundtrip_metadata( + tmp_path: pathlib.Path, +): dt = DeltaTable.create( tmp_path, - sample_data.schema, + schema, name="test_name", description="test_desc", configuration={ @@ -33,21 +49,21 @@ def test_create_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table assert {*dt.protocol().writer_features} == {"appendOnly", "timestampNtz"} # type: ignore -def test_create_modes(tmp_path: pathlib.Path, sample_data: pa.Table): - dt = DeltaTable.create(tmp_path, sample_data.schema, mode="error") +def test_create_modes(tmp_path: pathlib.Path): + dt = DeltaTable.create(tmp_path, schema, mode="error") last_action = dt.history(1)[0] with pytest.raises(DeltaError): - dt = DeltaTable.create(tmp_path, sample_data.schema, mode="error") + dt = DeltaTable.create(tmp_path, schema, mode="error") assert last_action["operation"] == "CREATE TABLE" with pytest.raises(DeltaError): - dt = DeltaTable.create(tmp_path, sample_data.schema, mode="append") + dt = DeltaTable.create(tmp_path, schema, mode="append") - dt = DeltaTable.create(tmp_path, sample_data.schema, mode="ignore") + dt = DeltaTable.create(tmp_path, schema, mode="ignore") assert dt.version() == 0 - dt = DeltaTable.create(tmp_path, sample_data.schema, mode="overwrite") + dt = DeltaTable.create(tmp_path, schema, mode="overwrite") assert dt.version() == 1 last_action = dt.history(1)[0] @@ -55,22 +71,21 @@ def test_create_modes(tmp_path: pathlib.Path, sample_data: pa.Table): assert last_action["operation"] == "CREATE OR REPLACE TABLE" -def test_create_schema(tmp_path: pathlib.Path, sample_data: pa.Table): +def test_create_schema(tmp_path: pathlib.Path): dt = DeltaTable.create( tmp_path, - sample_data.schema, + schema, ) - assert dt.schema().to_pyarrow() == sample_data.schema + assert dt.schema() == schema -def test_create_with_deletion_vectors_enabled( - tmp_path: pathlib.Path, sample_table: pa.Table -): +@pytest.mark.skip(reason="not implemented") +def test_create_with_deletion_vectors_enabled(tmp_path: pathlib.Path): """append only is set to false so shouldn't be converted to a feature""" dt = DeltaTable.create( tmp_path, - sample_table.schema, + schema, name="test_name", description="test_desc", configuration={ @@ -95,17 +110,23 @@ def test_create_with_deletion_vectors_enabled( assert dt.history()[0]["userName"] == "John Doe" -def test_create_higher_protocol_versions( - tmp_path: pathlib.Path, sample_table: pa.Table -): +def test_create_higher_protocol_versions(tmp_path: pathlib.Path): + schema = Schema( + fields=[ + Field("id", type=PrimitiveType("string"), nullable=True), + Field("price", type=PrimitiveType("integer"), nullable=True), + Field("deleted", type=PrimitiveType("boolean"), nullable=True), + ] + ) + dt = DeltaTable.create( tmp_path, - sample_table.schema, + schema, name="test_name", description="test_desc", configuration={ "delta.appendOnly": "false", - "delta.minReaderVersion": "2", + "delta.minReaderVersion": "1", "delta.minWriterVersion": "5", }, commit_properties=CommitProperties(custom_metadata={"userName": "John Doe"}), @@ -117,20 +138,21 @@ def test_create_higher_protocol_versions( assert metadata.description == "test_desc" assert metadata.configuration == { "delta.appendOnly": "false", - "delta.minReaderVersion": "2", + "delta.minReaderVersion": "1", "delta.minWriterVersion": "5", } - assert protocol.min_reader_version == 2 + assert protocol.min_reader_version == 1 assert protocol.min_writer_version == 5 assert dt.history()[0]["userName"] == "John Doe" -def test_create_or_replace_existing_table( - tmp_path: pathlib.Path, sample_data: pa.Table -): - write_deltalake(table_or_uri=tmp_path, data=sample_data) +def test_create_or_replace_existing_table(tmp_path: pathlib.Path, sample_table: Table): + write_deltalake(table_or_uri=tmp_path, data=sample_table) dt = DeltaTable.create( - tmp_path, sample_data.schema, partition_by=["utf8"], mode="overwrite" + tmp_path, + schema, + partition_by=["id"], + mode="overwrite", ) assert dt.files() == [] diff --git a/python/tests/test_delete.py b/python/tests/test_delete.py index 5ada819556..1650a276e2 100644 --- a/python/tests/test_delete.py +++ b/python/tests/test_delete.py @@ -1,40 +1,47 @@ import pathlib +from typing import TYPE_CHECKING -import pyarrow as pa -import pyarrow.compute as pc +import pytest +from arro3.core import Array, DataType, Field, Table from deltalake import CommitProperties, write_deltalake from deltalake.table import DeltaTable -from deltalake.writer._conversion import ( - ArrowSchemaConversionMode, - convert_pyarrow_table, -) +if TYPE_CHECKING: + import pyarrow as pa -def test_delete_no_predicates(existing_table: DeltaTable): - old_version = existing_table.version() + +def test_delete_no_predicates(existing_sample_table: DeltaTable): + old_version = existing_sample_table.version() commit_properties = CommitProperties(custom_metadata={"userName": "John Doe"}) - existing_table.delete(commit_properties=commit_properties) + existing_sample_table.delete(commit_properties=commit_properties) - last_action = existing_table.history(1)[0] + last_action = existing_sample_table.history(1)[0] assert last_action["operation"] == "DELETE" - assert existing_table.version() == old_version + 1 + assert existing_sample_table.version() == old_version + 1 assert last_action["userName"] == "John Doe" - dataset = existing_table.to_pyarrow_dataset() - assert dataset.count_rows() == 0 - assert len(existing_table.files()) == 0 + from deltalake.query import QueryBuilder + qb = QueryBuilder() + qb = QueryBuilder().register("tbl", existing_sample_table) + data = qb.execute("select * from tbl").read_all() -def test_delete_a_partition(tmp_path: pathlib.Path, sample_data: pa.Table): - write_deltalake(tmp_path, sample_data, partition_by=["bool"]) + assert data.num_rows == 0 + assert len(existing_sample_table.files()) == 0 + + +@pytest.mark.pyarrow +def test_delete_a_partition(tmp_path: pathlib.Path, sample_data_pyarrow: "pa.Table"): + write_deltalake(tmp_path, sample_data_pyarrow, partition_by=["bool"]) + import pyarrow.compute as pc dt = DeltaTable(tmp_path) old_version = dt.version() - mask = pc.equal(sample_data["bool"], False) - expected_table = sample_data.filter(mask) + mask = pc.equal(sample_data_pyarrow["bool"], False) + expected_table = sample_data_pyarrow.filter(mask) dt.delete(predicate="bool = true") @@ -47,7 +54,11 @@ def test_delete_a_partition(tmp_path: pathlib.Path, sample_data: pa.Table): assert len(dt.files()) == 1 +@pytest.mark.pyarrow def test_delete_some_rows(existing_table: DeltaTable): + import pyarrow as pa + import pyarrow.compute as pc + old_version = existing_table.version() existing = existing_table.to_pyarrow_table() @@ -64,35 +75,19 @@ def test_delete_some_rows(existing_table: DeltaTable): assert table.equals(expected_table) -def test_delete_large_dtypes(tmp_path: pathlib.Path, sample_table: pa.table): - sample_table = convert_pyarrow_table( - sample_table, schema_conversion_mode=ArrowSchemaConversionMode.LARGE - ) - write_deltalake(tmp_path, sample_table) # type: ignore - - dt = DeltaTable(tmp_path) - old_version = dt.version() - - existing = dt.to_pyarrow_table() - mask = pc.invert(pc.is_in(existing["id"], pa.array(["1"]))) - expected_table = existing.filter(mask) - - dt.delete(predicate="id = '1'") - - last_action = dt.history(1)[0] - assert last_action["operation"] == "DELETE" - assert dt.version() == old_version + 1 - - table = dt.to_pyarrow_table() - assert table.equals(expected_table) - - def test_delete_stats_columns_stats_provided(tmp_path: pathlib.Path): - data = pa.table( + data = Table( { - "foo": pa.array(["a", "b", None, None]), - "bar": pa.array([1, 2, 3, None]), - "baz": pa.array([1, 1, None, None]), + "foo": Array( + ["a", "b", None, None], + type=Field("foo", DataType.string(), nullable=True), + ), + "bar": Array( + [1, 2, 3, None], type=Field("bar", DataType.int64(), nullable=True) + ), + "baz": Array( + [1, 1, None, None], type=Field("baz", DataType.int64(), nullable=True) + ), } ) write_deltalake( @@ -103,31 +98,38 @@ def test_delete_stats_columns_stats_provided(tmp_path: pathlib.Path): ) dt = DeltaTable(tmp_path) add_actions_table = dt.get_add_actions(flatten=True) - stats = add_actions_table.to_pylist()[0] - - assert stats["null_count.foo"] == 2 - assert stats["min.foo"] == "a" - assert stats["max.foo"] == "b" - assert stats["null_count.bar"] is None - assert stats["min.bar"] is None - assert stats["max.bar"] is None - assert stats["null_count.baz"] == 2 - assert stats["min.baz"] == 1 - assert stats["max.baz"] == 1 + + def get_value(name: str): + return add_actions_table.column(name)[0].as_py() + + # x1 has no max, since inf was the highest value + assert get_value("null_count.foo") == 2 + assert get_value("min.foo") == "a" + assert get_value("max.foo") == "b" + assert get_value("null_count.bar") is None + assert get_value("min.bar") is None + assert get_value("max.bar") is None + assert get_value("null_count.baz") == 2 + assert get_value("min.baz") == 1 + assert get_value("max.baz") == 1 dt.delete("bar == 3") dt = DeltaTable(tmp_path) add_actions_table = dt.get_add_actions(flatten=True) - stats = add_actions_table.to_pylist()[0] assert dt.version() == 1 - assert stats["null_count.foo"] == 1 - assert stats["min.foo"] == "a" - assert stats["max.foo"] == "b" - assert stats["null_count.bar"] is None - assert stats["min.bar"] is None - assert stats["max.bar"] is None - assert stats["null_count.baz"] == 1 - assert stats["min.baz"] == 1 - assert stats["max.baz"] == 1 + + def get_value(name: str): + return add_actions_table.column(name)[0].as_py() + + # x1 has no max, since inf was the highest value + assert get_value("null_count.foo") == 1 + assert get_value("min.foo") == "a" + assert get_value("max.foo") == "b" + assert get_value("null_count.bar") is None + assert get_value("min.bar") is None + assert get_value("max.bar") is None + assert get_value("null_count.baz") == 1 + assert get_value("min.baz") == 1 + assert get_value("max.baz") == 1 diff --git a/python/tests/test_file_system_handler.py b/python/tests/test_file_system_handler.py index cd57d72dee..61e15a70b3 100644 --- a/python/tests/test_file_system_handler.py +++ b/python/tests/test_file_system_handler.py @@ -2,17 +2,15 @@ from pathlib import Path import numpy as np -import pyarrow as pa -import pyarrow.dataset as ds -import pyarrow.fs as fs -import pyarrow.parquet as pq import pytest -from deltalake.fs import DeltaStorageHandler - @pytest.fixture def file_systems(tmp_path: Path): + import pyarrow.fs as fs + + from deltalake.fs import DeltaStorageHandler + store = fs.PyFileSystem(DeltaStorageHandler(str(tmp_path.absolute()))) arrow_fs = fs.SubTreeFileSystem(str(tmp_path.absolute()), fs.LocalFileSystem()) return (store, arrow_fs) @@ -20,15 +18,20 @@ def file_systems(tmp_path: Path): @pytest.fixture def table_data(): + import pyarrow as pa + return pa.Table.from_arrays( [pa.array([1, 2, 3]), pa.array(["a", "b", "c"])], names=["int", "str"] ) +@pytest.mark.pyarrow def test_file_info(file_systems, table_data): store, arrow_fs = file_systems file_path = "table.parquet" + import pyarrow.parquet as pq + pq.write_table(table_data, file_path, filesystem=arrow_fs) info = store.get_file_info(file_path) @@ -42,8 +45,13 @@ def test_file_info(file_systems, table_data): assert info.mtime == arrow_info.mtime +@pytest.mark.pyarrow def test_get_file_info_selector(file_systems): store, arrow_fs = file_systems + import pyarrow as pa + import pyarrow.dataset as ds + import pyarrow.fs as fs + table = pa.table({"a": range(10), "b": np.random.randn(10), "c": [1, 2] * 5}) partitioning = ds.partitioning(pa.schema([("c", pa.int64())]), flavor="hive") ds.write_dataset( @@ -61,9 +69,11 @@ def test_get_file_info_selector(file_systems): assert Counter([i.type for i in infos]) == Counter([i.type for i in arrow_infos]) +@pytest.mark.pyarrow def test_open_input_file(file_systems, table_data): store, arrow_fs = file_systems file_path = "table.parquet" + import pyarrow.parquet as pq pq.write_table(table_data, file_path, filesystem=arrow_fs) @@ -99,9 +109,14 @@ def test_open_input_file(file_systems, table_data): assert file.read_at(10, 0) == arrow_file.read_at(10, 0) +@pytest.mark.pyarrow def test_open_input_file_with_size(tmp_path, table_data): file_path = "table.parquet" input_size = 12345 # incorrect file size for testing purposes + import pyarrow.fs as fs + import pyarrow.parquet as pq + + from deltalake.fs import DeltaStorageHandler # test that injected file size gets stored correctly store1 = DeltaStorageHandler( @@ -122,9 +137,12 @@ def test_open_input_file_with_size(tmp_path, table_data): assert file.size() != input_size +@pytest.mark.pyarrow def test_read_table(file_systems, table_data): store, arrow_fs = file_systems file_path = "table.parquet" + import pyarrow as pa + import pyarrow.parquet as pq pq.write_table(table_data, file_path, filesystem=arrow_fs) @@ -135,8 +153,13 @@ def test_read_table(file_systems, table_data): assert table.equals(arrow_table) +@pytest.mark.pyarrow def test_read_dataset(file_systems): store, arrow_fs = file_systems + import pyarrow as pa + import pyarrow.dataset as ds + import pyarrow.parquet as pq + table = pa.table({"a": range(10), "b": np.random.randn(10), "c": [1, 2] * 5}) pq.write_table(table.slice(0, 5), "data1.parquet", filesystem=arrow_fs) @@ -149,8 +172,13 @@ def test_read_dataset(file_systems): assert table.equals(ds_table) +@pytest.mark.pyarrow def test_write_table(file_systems): store, _ = file_systems + import pyarrow as pa + import pyarrow.dataset as ds + import pyarrow.parquet as pq + table = pa.table({"a": range(10), "b": np.random.randn(10), "c": [1, 2] * 5}) pq.write_table(table.slice(0, 5), "data1.parquet", filesystem=store) @@ -163,8 +191,12 @@ def test_write_table(file_systems): assert table.equals(ds_table) +@pytest.mark.pyarrow def test_write_partitioned_dataset(file_systems): store, arrow_fs = file_systems + import pyarrow as pa + import pyarrow.dataset as ds + table = pa.table({"a": range(10), "b": np.random.randn(10), "c": [1, 2] * 5}) partitioning = ds.partitioning(pa.schema([("c", pa.int64())]), flavor="hive") diff --git a/python/tests/test_fs.py b/python/tests/test_fs.py index b08befc735..45e8c5e7e2 100644 --- a/python/tests/test_fs.py +++ b/python/tests/test_fs.py @@ -1,17 +1,26 @@ import pickle import urllib +from typing import TYPE_CHECKING -import pyarrow as pa -import pyarrow.parquet as pq import pytest -from pyarrow.fs import FileType from deltalake import DeltaTable from deltalake.exceptions import DeltaProtocolError from deltalake.fs import DeltaStorageHandler from deltalake.writer import write_deltalake +if TYPE_CHECKING: + import pyarrow as pa +try: + import pyarrow as pa + import pyarrow.parquet as pq + from pyarrow.fs import FileType +except ImportError: + pass + + +@pytest.mark.pyarrow @pytest.mark.s3 @pytest.mark.integration @pytest.mark.timeout(timeout=15, method="thread") @@ -29,6 +38,7 @@ def test_read_files(s3_localstack): assert table.shape > (0, 0) +@pytest.mark.pyarrow @pytest.mark.s3 @pytest.mark.integration @pytest.mark.timeout(timeout=15, method="thread") @@ -42,6 +52,7 @@ def test_read_file_info(s3_localstack): assert meta[0].type == FileType.File +@pytest.mark.pyarrow @pytest.mark.s3 @pytest.mark.integration @pytest.mark.timeout(timeout=15, method="thread") @@ -61,13 +72,14 @@ def test_s3_authenticated_read_write(s3_localstack_creds, monkeypatch): # Make a get request on an object with pytest.raises(Exception): - storage_handler.open_input_stream("nonexistant") + storage_handler.open_input_stream("nonexistent") # Try to write an object with pytest.raises(Exception): - storage_handler.open_output_stream("nonexistant") + storage_handler.open_output_stream("nonexistent") +@pytest.mark.pyarrow @pytest.mark.s3 @pytest.mark.integration @pytest.mark.timeout(timeout=15, method="thread") @@ -88,41 +100,43 @@ def test_read_simple_table_from_remote(s3_localstack): assert dt.file_uris() == [table_path + "/" + path for path in expected_files] +@pytest.mark.pyarrow @pytest.mark.s3 @pytest.mark.integration @pytest.mark.timeout(timeout=15, method="thread") @pytest.mark.skip( reason="Temporarily disabled until we can resolve https://github.com/delta-io/delta-rs/pull/2120#issuecomment-1912367573" ) -def test_roundtrip_s3_env(s3_localstack, sample_data: pa.Table, monkeypatch): +def test_roundtrip_s3_env(s3_localstack, sample_data_pyarrow: "pa.Table", monkeypatch): table_path = "s3://deltars/roundtrip" # Create new table with path with pytest.raises(DeltaProtocolError, match="Atomic rename requires a LockClient"): - write_deltalake(table_path, sample_data) + write_deltalake(table_path, sample_data_pyarrow) monkeypatch.setenv("AWS_S3_ALLOW_UNSAFE_RENAME", "true") # Create new table with path - write_deltalake(table_path, sample_data) + write_deltalake(table_path, sample_data_pyarrow) dt = DeltaTable(table_path) table = dt.to_pyarrow_table() - assert table == sample_data + assert table == sample_data_pyarrow assert dt.version() == 0 # Write with existing DeltaTable - write_deltalake(dt, sample_data, mode="overwrite") + write_deltalake(dt, sample_data_pyarrow, mode="overwrite") dt.update_incremental() assert dt.version() == 1 table = dt.to_pyarrow_table() - assert table == sample_data + assert table == sample_data_pyarrow +@pytest.mark.pyarrow @pytest.mark.s3 @pytest.mark.integration @pytest.mark.timeout(timeout=15, method="thread") -def test_roundtrip_s3_direct(s3_localstack_creds, sample_data: pa.Table): +def test_roundtrip_s3_direct(s3_localstack_creds, sample_data_pyarrow: "pa.Table"): table_path = "s3://deltars/roundtrip2" # Fails without any credentials @@ -136,7 +150,7 @@ def test_roundtrip_s3_direct(s3_localstack_creds, sample_data: pa.Table): } write_deltalake( table_path, - sample_data, + sample_data_pyarrow, storage_options=anon_storage_options, ) @@ -146,92 +160,99 @@ def test_roundtrip_s3_direct(s3_localstack_creds, sample_data: pa.Table): "AWS_S3_ALLOW_UNSAFE_RENAME": "true", } storage_opts.update(s3_localstack_creds) - write_deltalake(table_path, sample_data, storage_options=storage_opts) + write_deltalake(table_path, sample_data_pyarrow, storage_options=storage_opts) dt = DeltaTable(table_path, storage_options=storage_opts) assert dt.version() == 0 table = dt.to_pyarrow_table() - assert table == sample_data + assert table == sample_data_pyarrow # Can pass storage_options into DeltaTable and then write - write_deltalake(dt, sample_data, mode="overwrite") + write_deltalake(dt, sample_data_pyarrow, mode="overwrite") dt.update_incremental() assert dt.version() == 1 table = dt.to_pyarrow_table() - assert table == sample_data + assert table == sample_data_pyarrow +@pytest.mark.pyarrow @pytest.mark.azure @pytest.mark.integration @pytest.mark.timeout(timeout=60, method="thread") -def test_roundtrip_azure_env(azurite_env_vars, sample_data: pa.Table): +def test_roundtrip_azure_env(azurite_env_vars, sample_data_pyarrow: "pa.Table"): table_path = "az://deltars/roundtrip" # Create new table with path - write_deltalake(table_path, sample_data) + write_deltalake(table_path, sample_data_pyarrow) dt = DeltaTable(table_path) table = dt.to_pyarrow_table() - assert table == sample_data + assert table == sample_data_pyarrow assert dt.version() == 0 # Write with existing DeltaTable - write_deltalake(dt, sample_data, mode="overwrite") + write_deltalake(dt, sample_data_pyarrow, mode="overwrite") dt.update_incremental() assert dt.version() == 1 table = dt.to_pyarrow_table() - assert table == sample_data + assert table == sample_data_pyarrow +@pytest.mark.pyarrow @pytest.mark.azure @pytest.mark.integration @pytest.mark.timeout(timeout=60, method="thread") -def test_roundtrip_azure_direct(azurite_creds, sample_data: pa.Table): +def test_roundtrip_azure_direct(azurite_creds, sample_data_pyarrow: "pa.Table"): table_path = "az://deltars/roundtrip2" # Can pass storage_options in directly - write_deltalake(table_path, sample_data, storage_options=azurite_creds) + write_deltalake(table_path, sample_data_pyarrow, storage_options=azurite_creds) dt = DeltaTable(table_path, storage_options=azurite_creds) table = dt.to_pyarrow_table() - assert table == sample_data + assert table == sample_data_pyarrow assert dt.version() == 0 # Can pass storage_options into DeltaTable and then write - write_deltalake(dt, sample_data, mode="overwrite") + write_deltalake(dt, sample_data_pyarrow, mode="overwrite") dt.update_incremental() assert dt.version() == 1 table = dt.to_pyarrow_table() - assert table == sample_data + assert table == sample_data_pyarrow +@pytest.mark.pyarrow @pytest.mark.azure @pytest.mark.integration @pytest.mark.timeout(timeout=60, method="thread") -def test_roundtrip_azure_sas(azurite_sas_creds, sample_data: pa.Table): +def test_roundtrip_azure_sas(azurite_sas_creds, sample_data_pyarrow: "pa.Table"): table_path = "az://deltars/roundtrip3" - write_deltalake(table_path, sample_data, storage_options=azurite_sas_creds) + write_deltalake(table_path, sample_data_pyarrow, storage_options=azurite_sas_creds) dt = DeltaTable(table_path, storage_options=azurite_sas_creds) table = dt.to_pyarrow_table() - assert table == sample_data + assert table == sample_data_pyarrow assert dt.version() == 0 +@pytest.mark.pyarrow @pytest.mark.azure @pytest.mark.integration @pytest.mark.timeout(timeout=60, method="thread") -def test_roundtrip_azure_decoded_sas(azurite_sas_creds, sample_data: pa.Table): +def test_roundtrip_azure_decoded_sas( + azurite_sas_creds, sample_data_pyarrow: "pa.Table" +): table_path = "az://deltars/roundtrip4" azurite_sas_creds["SAS_TOKEN"] = urllib.parse.unquote( azurite_sas_creds["SAS_TOKEN"] ) - write_deltalake(table_path, sample_data, storage_options=azurite_sas_creds) + write_deltalake(table_path, sample_data_pyarrow, storage_options=azurite_sas_creds) dt = DeltaTable(table_path, storage_options=azurite_sas_creds) table = dt.to_pyarrow_table() - assert table == sample_data + assert table == sample_data_pyarrow assert dt.version() == 0 +@pytest.mark.pyarrow @pytest.mark.parametrize("storage_size", [1, 4 * 1024 * 1024, 5 * 1024 * 1024 - 1]) def test_warning_for_small_max_buffer_size(tmp_path, storage_size): storage_opts = {"max_buffer_size": str(storage_size)} @@ -246,6 +267,7 @@ def test_warning_for_small_max_buffer_size(tmp_path, storage_size): ) +@pytest.mark.pyarrow def test_pickle_roundtrip(tmp_path): store = DeltaStorageHandler(str(tmp_path.absolute())) diff --git a/python/tests/test_generated_columns.py b/python/tests/test_generated_columns.py index db9395db37..18a85de66b 100644 --- a/python/tests/test_generated_columns.py +++ b/python/tests/test_generated_columns.py @@ -1,19 +1,28 @@ -import pyarrow as pa +from typing import TYPE_CHECKING + import pytest +from arro3.core import Array, DataType, Table +from arro3.core import Field as ArrowField -from deltalake import DeltaTable, Field, Schema, write_deltalake +from deltalake import DeltaTable, Field, write_deltalake +from deltalake import Schema as DeltaSchema from deltalake.exceptions import DeltaError, SchemaMismatchError +from deltalake.query import QueryBuilder from deltalake.schema import PrimitiveType +if TYPE_CHECKING: + import pyarrow as pa + @pytest.fixture -def gc_schema() -> Schema: - return Schema( +def gc_schema() -> DeltaSchema: + return DeltaSchema( [ - Field(name="id", type=PrimitiveType("integer")), + Field(name="id", type=PrimitiveType("integer"), nullable=True), Field( name="gc", type=PrimitiveType("integer"), + nullable=True, metadata={"delta.generationExpression": "5"}, ), ] @@ -21,28 +30,32 @@ def gc_schema() -> Schema: @pytest.fixture -def valid_gc_data() -> pa.Table: - id_col = pa.field("id", pa.int32()) - gc = pa.field("gc", pa.int32()).with_metadata({"delta.generationExpression": "10"}) - data = pa.Table.from_pydict( - {"id": [1, 2], "gc": [10, 10]}, schema=pa.schema([id_col, gc]) +def valid_gc_data() -> Table: + id_col = ArrowField("id", DataType.int32(), nullable=True) + gc = ArrowField("gc", DataType.int32(), nullable=True).with_metadata( + {"delta.generationExpression": "10"} + ) + data = Table.from_pydict( + {"id": Array([1, 2], type=id_col), "gc": Array([10, 10], type=gc)}, ) return data @pytest.fixture -def data_without_gc() -> pa.Table: - id_col = pa.field("id", pa.int32()) - data = pa.Table.from_pydict({"id": [1, 2]}, schema=pa.schema([id_col])) +def data_without_gc() -> Table: + id_col = ArrowField("id", DataType.int32(), nullable=True) + data = Table.from_pydict({"id": Array([1, 2], type=id_col)}) return data @pytest.fixture -def invalid_gc_data() -> pa.Table: - id_col = pa.field("id", pa.int32()) - gc = pa.field("gc", pa.int32()).with_metadata({"delta.generationExpression": "10"}) - data = pa.Table.from_pydict( - {"id": [1, 2], "gc": [5, 10]}, schema=pa.schema([id_col, gc]) +def invalid_gc_data() -> "pa.Table": + id_col = ArrowField("id", DataType.int32(), nullable=True) + gc = ArrowField("gc", DataType.int32(), nullable=True).with_metadata( + {"delta.generationExpression": "10"} + ) + data = Table.from_pydict( + {"id": Array([1, 2], type=id_col), "gc": Array([5, 10], type=gc)}, ) return data @@ -56,7 +69,7 @@ def table_with_gc(tmp_path, gc_schema) -> DeltaTable: return dt -def test_create_table_with_generated_columns(tmp_path, gc_schema: Schema): +def test_create_table_with_generated_columns(tmp_path, gc_schema: DeltaSchema): dt = DeltaTable.create( tmp_path, schema=gc_schema, @@ -82,7 +95,11 @@ def test_write_with_gc(tmp_path, valid_gc_data): dt = DeltaTable(tmp_path) assert dt.protocol().min_writer_version == 4 - assert dt.to_pyarrow_table() == valid_gc_data + + from deltalake.query import QueryBuilder + + data = QueryBuilder().register("tbl", dt).execute("select * from tbl").read_all() + assert data == valid_gc_data def test_write_with_gc_higher_writer_version(tmp_path, valid_gc_data): @@ -97,7 +114,10 @@ def test_write_with_gc_higher_writer_version(tmp_path, valid_gc_data): assert protocol.min_writer_version == 7 assert protocol.writer_features is not None assert "generatedColumns" in protocol.writer_features - assert dt.to_pyarrow_table() == valid_gc_data + assert ( + QueryBuilder().register("tbl", dt).execute("select * from tbl").read_all() + == valid_gc_data + ) def test_write_with_invalid_gc(tmp_path, invalid_gc_data): @@ -106,7 +126,7 @@ def test_write_with_invalid_gc(tmp_path, invalid_gc_data): with pytest.raises( DeltaError, match=re.escape( - 'Invariant violations: ["Check or Invariant (gc = 10 OR (gc IS NULL AND 10 IS NULL)) violated by value in row: [5]"]' + 'Invariant violations: ["Check or Invariant (gc <=> 10) violated by value in row: [5]"]' ), ): write_deltalake(tmp_path, mode="append", data=invalid_gc_data) @@ -118,25 +138,33 @@ def test_write_with_invalid_gc_to_table(table_with_gc, invalid_gc_data): with pytest.raises( DeltaError, match=re.escape( - 'Invariant violations: ["Check or Invariant (gc = 5 OR (gc IS NULL AND 5 IS NULL)) violated by value in row: [10]"]' + 'Invariant violations: ["Check or Invariant (gc <=> 5) violated by value in row: [10]"]' ), ): write_deltalake(table_with_gc, mode="append", data=invalid_gc_data) def test_write_to_table_generating_data(table_with_gc: DeltaTable): - id_col = pa.field("id", pa.int32()) - data = pa.Table.from_pydict({"id": [1, 2]}, schema=pa.schema([id_col])) + id_col = ArrowField("id", DataType.int32(), nullable=True) + + data = Table.from_pydict({"id": Array([1, 2], type=id_col)}) write_deltalake(table_with_gc, mode="append", data=data) - id_col = pa.field("id", pa.int32()) - gc = pa.field("gc", pa.int32()) - expected_data = pa.Table.from_pydict( - {"id": [1, 2], "gc": [5, 5]}, schema=pa.schema([id_col, gc]) + gc = ArrowField("gc", DataType.int32(), nullable=True).with_metadata( + {"delta.generationExpression": "5"} + ) + expected_data = Table.from_pydict( + {"id": Array([1, 2], type=id_col), "gc": Array([5, 5], type=gc)}, ) assert table_with_gc.version() == 1 - assert table_with_gc.to_pyarrow_table() == expected_data + result = ( + QueryBuilder() + .register("tbl", table_with_gc) + .execute("select * from tbl order by id asc") + .read_all() + ) + assert result.schema == expected_data.schema def test_raise_when_gc_passed_during_schema_evolution( @@ -191,15 +219,25 @@ def test_merge_with_gc(table_with_gc: DeltaTable, data_without_gc): .when_not_matched_insert_all() .execute() ) - id_col = pa.field("id", pa.int32()) - gc = pa.field("gc", pa.int32()) - expected_data = pa.Table.from_pydict( - {"id": [1, 2], "gc": [5, 5]}, schema=pa.schema([id_col, gc]) + + id_col = ArrowField("id", DataType.int32(), nullable=True) + + gc = ArrowField("gc", DataType.int32(), nullable=True).with_metadata( + {"delta.generationExpression": "5"} ) - assert ( - table_with_gc.to_pyarrow_table().sort_by([("id", "ascending")]) == expected_data + expected_data = Table.from_pydict( + {"id": Array([1, 2], type=id_col), "gc": Array([5, 5], type=gc)}, + ) + + result = ( + QueryBuilder() + .register("tbl", table_with_gc) + .execute("select * from tbl order by id asc") + .read_all() ) + assert result == expected_data + def test_merge_with_g_during_schema_evolution( table_with_gc: DeltaTable, data_without_gc @@ -215,14 +253,20 @@ def test_merge_with_g_during_schema_evolution( .when_not_matched_insert_all() .execute() ) - id_col = pa.field("id", pa.int32()) - gc = pa.field("gc", pa.int32()) - expected_data = pa.Table.from_pydict( - {"id": [1, 2], "gc": [5, 5]}, schema=pa.schema([id_col, gc]) + + id_col = ArrowField("id", DataType.int32(), nullable=True) + gc = ArrowField("gc", DataType.int32(), nullable=True) + expected_data = Table.from_pydict( + {"id": Array([1, 2], type=id_col), "gc": Array([5, 5], type=gc)}, ) - assert ( - table_with_gc.to_pyarrow_table().sort_by([("id", "ascending")]) == expected_data + + result = ( + QueryBuilder() + .register("tbl", table_with_gc) + .execute("select * from tbl order by id asc") + .read_all() ) + assert result == expected_data def test_raise_when_gc_passed_merge_statement_during_schema_evolution( @@ -259,7 +303,7 @@ def test_merge_with_gc_invalid(table_with_gc: DeltaTable, invalid_gc_data): with pytest.raises( DeltaError, match=re.escape( - 'Invariant violations: ["Check or Invariant (gc = 5 OR (gc IS NULL AND 5 IS NULL)) violated by value in row: [10]"]' + 'Invariant violations: ["Check or Invariant (gc <=> 5) violated by value in row: [10]"]' ), ): ( diff --git a/python/tests/test_lakefs.py b/python/tests/test_lakefs.py index 244cdd4b74..01fc8e3a63 100644 --- a/python/tests/test_lakefs.py +++ b/python/tests/test_lakefs.py @@ -3,12 +3,14 @@ from datetime import timedelta from typing import TYPE_CHECKING -import pyarrow as pa import pytest +from arro3.core import Array, DataType, Table +from arro3.core import Field as ArrowField from deltalake import CommitProperties, DeltaTable, TableFeatures -from deltalake._internal import Field, PrimitiveType +from deltalake._internal import Field, PrimitiveType, Schema from deltalake.exceptions import DeltaError, DeltaProtocolError +from deltalake.query import QueryBuilder from deltalake.writer import write_deltalake from tests.test_alter import _sort_fields @@ -16,6 +18,17 @@ import lakefs +@pytest.fixture +def delta_schema() -> Schema: + return Schema( + fields=[ + Field("date", PrimitiveType("date")), + Field("foo", PrimitiveType("string")), + Field("bar", PrimitiveType("string")), + ] + ) + + @pytest.fixture def lakefs_client(): import lakefs @@ -42,10 +55,10 @@ def lakefs_storage_options(): @pytest.mark.lakefs @pytest.mark.integration -def test_create(lakefs_path: str, sample_data: pa.Table, lakefs_storage_options): +def test_create(lakefs_path: str, delta_schema: Schema, lakefs_storage_options): dt = DeltaTable.create( lakefs_path, - sample_data.schema, + delta_schema, mode="error", storage_options=lakefs_storage_options, ) @@ -54,7 +67,7 @@ def test_create(lakefs_path: str, sample_data: pa.Table, lakefs_storage_options) with pytest.raises(DeltaError): dt = DeltaTable.create( lakefs_path, - sample_data.schema, + delta_schema, mode="error", storage_options=lakefs_storage_options, ) @@ -63,14 +76,14 @@ def test_create(lakefs_path: str, sample_data: pa.Table, lakefs_storage_options) with pytest.raises(DeltaError): dt = DeltaTable.create( lakefs_path, - sample_data.schema, + delta_schema, mode="append", storage_options=lakefs_storage_options, ) dt = DeltaTable.create( lakefs_path, - sample_data.schema, + delta_schema, mode="ignore", storage_options=lakefs_storage_options, ) @@ -78,7 +91,7 @@ def test_create(lakefs_path: str, sample_data: pa.Table, lakefs_storage_options) dt = DeltaTable.create( lakefs_path, - sample_data.schema, + delta_schema, mode="overwrite", storage_options=lakefs_storage_options, ) @@ -91,11 +104,11 @@ def test_create(lakefs_path: str, sample_data: pa.Table, lakefs_storage_options) @pytest.mark.lakefs @pytest.mark.integration -def test_delete(lakefs_path: str, sample_data: pa.Table, lakefs_storage_options): +def test_delete(lakefs_path: str, sample_table: Table, lakefs_storage_options): write_deltalake( lakefs_path, - sample_data, - partition_by=["bool"], + sample_table, + partition_by=["id"], storage_options=lakefs_storage_options, ) @@ -109,35 +122,35 @@ def test_delete(lakefs_path: str, sample_data: pa.Table, lakefs_storage_options) assert dt.version() == old_version + 1 assert last_action["userName"] == "John Doe" - dataset = dt.to_pyarrow_dataset() - assert dataset.count_rows() == 0 + dataset = QueryBuilder().register("tbl", dt).execute("select * from tbl").read_all() + assert dataset.num_rows == 0 assert len(dt.files()) == 0 @pytest.mark.lakefs @pytest.mark.integration def test_optimize_min_commit_interval( - lakefs_path: str, sample_data: pa.Table, lakefs_storage_options + lakefs_path: str, sample_table: Table, lakefs_storage_options ): print(lakefs_path) write_deltalake( lakefs_path, - sample_data, - partition_by="utf8", + sample_table, + partition_by="id", mode="append", storage_options=lakefs_storage_options, ) write_deltalake( lakefs_path, - sample_data, - partition_by="utf8", + sample_table, + partition_by="id", mode="append", storage_options=lakefs_storage_options, ) write_deltalake( lakefs_path, - sample_data, - partition_by="utf8", + sample_table, + partition_by="id", mode="append", storage_options=lakefs_storage_options, ) @@ -145,7 +158,7 @@ def test_optimize_min_commit_interval( dt = DeltaTable(lakefs_path, storage_options=lakefs_storage_options) old_version = dt.version() - dt.optimize.z_order(["date32", "timestamp"], min_commit_interval=timedelta(0)) + dt.optimize.z_order(["sold", "price"], min_commit_interval=timedelta(0)) last_action = dt.history(1)[0] assert last_action["operation"] == "OPTIMIZE" @@ -157,25 +170,25 @@ def test_optimize_min_commit_interval( @pytest.mark.lakefs @pytest.mark.integration -def test_optimize(lakefs_path: str, sample_data: pa.Table, lakefs_storage_options): +def test_optimize(lakefs_path: str, sample_table: Table, lakefs_storage_options): write_deltalake( lakefs_path, - sample_data, - partition_by="utf8", + sample_table, + partition_by="id", mode="append", storage_options=lakefs_storage_options, ) write_deltalake( lakefs_path, - sample_data, - partition_by="utf8", + sample_table, + partition_by="id", mode="append", storage_options=lakefs_storage_options, ) write_deltalake( lakefs_path, - sample_data, - partition_by="utf8", + sample_table, + partition_by="id", mode="append", storage_options=lakefs_storage_options, ) @@ -183,7 +196,7 @@ def test_optimize(lakefs_path: str, sample_data: pa.Table, lakefs_storage_option dt = DeltaTable(lakefs_path, storage_options=lakefs_storage_options) old_version = dt.version() - dt.optimize.z_order(["date32", "timestamp"]) + dt.optimize.z_order(["sold", "price"]) last_action = dt.history(1)[0] assert last_action["operation"] == "OPTIMIZE" @@ -196,15 +209,24 @@ def test_optimize(lakefs_path: str, sample_data: pa.Table, lakefs_storage_option @pytest.mark.lakefs @pytest.mark.integration def test_repair_wo_dry_run( - lakefs_path, sample_data, lakefs_storage_options, lakefs_client: "lakefs.Client" + lakefs_path, + sample_table, + lakefs_storage_options, + lakefs_client: "lakefs.Client", ): import lakefs write_deltalake( - lakefs_path, sample_data, mode="append", storage_options=lakefs_storage_options + lakefs_path, + sample_table, + mode="append", + storage_options=lakefs_storage_options, ) write_deltalake( - lakefs_path, sample_data, mode="append", storage_options=lakefs_storage_options + lakefs_path, + sample_table, + mode="append", + storage_options=lakefs_storage_options, ) dt = DeltaTable(lakefs_path, storage_options=lakefs_storage_options) @@ -226,7 +248,7 @@ def test_repair_wo_dry_run( @pytest.mark.lakefs @pytest.mark.integration -def test_add_constraint(lakefs_path, sample_table: pa.Table, lakefs_storage_options): +def test_add_constraint(lakefs_path, sample_table: Table, lakefs_storage_options): write_deltalake(lakefs_path, sample_table, storage_options=lakefs_storage_options) dt = DeltaTable(lakefs_path, storage_options=lakefs_storage_options) @@ -244,12 +266,22 @@ def test_add_constraint(lakefs_path, sample_table: pa.Table, lakefs_storage_opti dt.alter.add_constraint({"check_price": "price < 0"}) with pytest.raises(DeltaProtocolError): - data = pa.table( + data = Table( { - "id": pa.array(["1"]), - "price": pa.array([-1], pa.int64()), - "sold": pa.array(list(range(1)), pa.int32()), - "deleted": pa.array([False] * 1), + "id": Array( + ["1"], type=ArrowField("id", type=DataType.string(), nullable=True) + ), + "price": Array( + [-1], type=ArrowField("price", type=DataType.int64(), nullable=True) + ), + "sold": Array( + list(range(1)), + type=ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False], + type=ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) write_deltalake( @@ -262,7 +294,7 @@ def test_add_constraint(lakefs_path, sample_table: pa.Table, lakefs_storage_opti @pytest.mark.lakefs @pytest.mark.integration -def test_drop_constraint(lakefs_path, sample_table: pa.Table, lakefs_storage_options): +def test_drop_constraint(lakefs_path, sample_table: Table, lakefs_storage_options): write_deltalake(lakefs_path, sample_table, storage_options=lakefs_storage_options) dt = DeltaTable(lakefs_path, storage_options=lakefs_storage_options) @@ -276,9 +308,7 @@ def test_drop_constraint(lakefs_path, sample_table: pa.Table, lakefs_storage_opt @pytest.mark.lakefs @pytest.mark.integration -def test_set_table_properties( - lakefs_path, sample_table: pa.Table, lakefs_storage_options -): +def test_set_table_properties(lakefs_path, sample_table: Table, lakefs_storage_options): write_deltalake( lakefs_path, sample_table, @@ -296,7 +326,7 @@ def test_set_table_properties( @pytest.mark.lakefs @pytest.mark.integration -def test_add_feautres(lakefs_path, sample_table: pa.Table, lakefs_storage_options): +def test_add_features(lakefs_path, sample_table: Table, lakefs_storage_options): write_deltalake(lakefs_path, sample_table, storage_options=lakefs_storage_options) dt = DeltaTable(lakefs_path, storage_options=lakefs_storage_options) dt.alter.add_feature( @@ -323,17 +353,19 @@ def test_add_feautres(lakefs_path, sample_table: pa.Table, lakefs_storage_option @pytest.mark.lakefs @pytest.mark.integration -def test_merge(lakefs_path, sample_table: pa.Table, lakefs_storage_options): +def test_merge(lakefs_path, sample_table: Table, lakefs_storage_options): write_deltalake( lakefs_path, sample_table, mode="append", storage_options=lakefs_storage_options ) dt = DeltaTable(lakefs_path, storage_options=lakefs_storage_options) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["5"]), - "weight": pa.array([105], pa.int32()), + "id": Array(["5"], type=ArrowField("id", DataType.string(), nullable=True)), + "weight": Array( + [105], type=ArrowField("weight", DataType.int32(), nullable=True) + ), } ) @@ -347,15 +379,29 @@ def test_merge(lakefs_path, sample_table: pa.Table, lakefs_storage_options): ).when_matched_delete().execute() nrows = 4 - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4"]), - "price": pa.array(list(range(nrows)), pa.int64()), - "sold": pa.array(list(range(nrows)), pa.int32()), - "deleted": pa.array([False] * nrows), + "id": Array( + ["1", "2", "3", "4"], + type=ArrowField("id", DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 3], + type=ArrowField("price", DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 3], + type=ArrowField("sold", DataType.int32(), nullable=True), + ), + "deleted": Array( + [False] * nrows, + type=ArrowField("deleted", DataType.bool(), nullable=True), + ), } ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + + result = QueryBuilder().register("tbl", dt).execute("select * from tbl").read_all() + last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -367,17 +413,26 @@ def test_merge(lakefs_path, sample_table: pa.Table, lakefs_storage_options): @pytest.mark.integration def test_restore( lakefs_path, - sample_data: pa.Table, + sample_table: Table, lakefs_storage_options, ): write_deltalake( - lakefs_path, sample_data, mode="append", storage_options=lakefs_storage_options + lakefs_path, + sample_table, + mode="append", + storage_options=lakefs_storage_options, ) write_deltalake( - lakefs_path, sample_data, mode="append", storage_options=lakefs_storage_options + lakefs_path, + sample_table, + mode="append", + storage_options=lakefs_storage_options, ) write_deltalake( - lakefs_path, sample_data, mode="append", storage_options=lakefs_storage_options + lakefs_path, + sample_table, + mode="append", + storage_options=lakefs_storage_options, ) dt = DeltaTable(lakefs_path, storage_options=lakefs_storage_options) @@ -392,9 +447,12 @@ def test_restore( @pytest.mark.lakefs @pytest.mark.integration -def test_add_column(lakefs_path, sample_data: pa.Table, lakefs_storage_options): +def test_add_column(lakefs_path, sample_table: Table, lakefs_storage_options): write_deltalake( - lakefs_path, sample_data, mode="append", storage_options=lakefs_storage_options + lakefs_path, + sample_table, + mode="append", + storage_options=lakefs_storage_options, ) dt = DeltaTable(lakefs_path, storage_options=lakefs_storage_options) current_fields = dt.schema().fields @@ -415,21 +473,35 @@ def test_add_column(lakefs_path, sample_data: pa.Table, lakefs_storage_options): @pytest.fixture() def sample_table_update(): nrows = 5 - return pa.table( + return Table( { - "id": pa.array(["1", "2", "3", "4", "5"]), - "price": pa.array(list(range(nrows)), pa.int64()), - "sold": pa.array(list(range(nrows)), pa.int64()), - "price_float": pa.array(list(range(nrows)), pa.float64()), - "items_in_bucket": pa.array([["item1", "item2", "item3"]] * nrows), - "deleted": pa.array([False] * nrows), + "id": Array( + ["1", "2", "3", "4", "5"], + type=ArrowField("id", DataType.string(), nullable=True), + ), + "price": Array( + list(range(nrows)), + type=ArrowField("price", DataType.int64(), nullable=True), + ), + "sold": Array( + list(range(nrows)), + type=ArrowField("sold", DataType.int64(), nullable=True), + ), + "price_float": Array( + list(range(nrows)), + type=ArrowField("price_float", DataType.float64(), nullable=True), + ), + "deleted": Array( + [False] * nrows, + type=ArrowField("deleted", DataType.bool(), nullable=True), + ), } ) @pytest.mark.lakefs @pytest.mark.integration -def test_update(lakefs_path, sample_table_update: pa.Table, lakefs_storage_options): +def test_update(lakefs_path, sample_table_update: Table, lakefs_storage_options): write_deltalake( lakefs_path, sample_table_update, @@ -439,14 +511,29 @@ def test_update(lakefs_path, sample_table_update: pa.Table, lakefs_storage_optio dt = DeltaTable(lakefs_path, storage_options=lakefs_storage_options) nrows = 5 - expected = pa.table( + + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5"]), - "price": pa.array(list(range(nrows)), pa.int64()), - "sold": pa.array(list(range(nrows)), pa.int64()), - "price_float": pa.array(list(range(nrows)), pa.float64()), - "items_in_bucket": pa.array([["item1", "item2", "item3"]] * nrows), - "deleted": pa.array([False, False, False, False, True]), + "id": Array( + ["1", "2", "3", "4", "5"], + type=ArrowField("id", DataType.string(), nullable=True), + ), + "price": Array( + list(range(nrows)), + type=ArrowField("price", DataType.int64(), nullable=True), + ), + "sold": Array( + list(range(nrows)), + type=ArrowField("sold", DataType.int64(), nullable=True), + ), + "price_float": Array( + list(range(nrows)), + type=ArrowField("price_float", DataType.float64(), nullable=True), + ), + "deleted": Array( + [False, False, False, False, True], + type=ArrowField("deleted", DataType.bool(), nullable=True), + ), } ) @@ -457,7 +544,7 @@ def test_update(lakefs_path, sample_table_update: pa.Table, lakefs_storage_optio commit_properties=commit_properties, ) - result = dt.to_pyarrow_table() + result = QueryBuilder().register("tbl", dt).execute("select * from tbl").read_all() last_action = dt.history(1)[0] assert last_action["operation"] == "UPDATE" @@ -467,7 +554,7 @@ def test_update(lakefs_path, sample_table_update: pa.Table, lakefs_storage_optio @pytest.mark.lakefs @pytest.mark.integration -def test_checkpoint(sample_data: pa.Table, lakefs_storage_options, lakefs_client): +def test_checkpoint(sample_table: Table, lakefs_storage_options, lakefs_client): import lakefs table = str(uuid.uuid4()) @@ -481,10 +568,8 @@ def test_checkpoint(sample_data: pa.Table, lakefs_storage_options, lakefs_client repository_id="bronze", branch_id="main", client=lakefs_client ) - # TODO: Include binary after fixing issue "Json error: binary type is not supported" - sample_data = sample_data.drop(["binary"]) write_deltalake( - str(tmp_table_path), sample_data, storage_options=lakefs_storage_options + str(tmp_table_path), sample_table, storage_options=lakefs_storage_options ) assert not branch.object(checkpoint_path).exists() @@ -500,13 +585,44 @@ def test_checkpoint(sample_data: pa.Table, lakefs_storage_options, lakefs_client @pytest.mark.lakefs @pytest.mark.integration -def test_storage_options(sample_data: pa.Table): +def test_no_empty_commits( + lakefs_path, sample_table: Table, lakefs_storage_options, lakefs_client +): + import lakefs + + write_deltalake( + lakefs_path, + sample_table, + mode="append", + storage_options=lakefs_storage_options, + ) + dt = DeltaTable(lakefs_path, storage_options=lakefs_storage_options) + + # Get current branch head commit before operation + branch = lakefs.Branch( + repository_id="bronze", branch_id="main", client=lakefs_client + ) + commits_before = list(branch.log()) + before_commit_id = commits_before[0].id if commits_before else None + + # Since there should be no files to vacuum in a fresh table this should be a no-op operation + dt.vacuum(dry_run=False) + + commits_after = list(branch.log()) + after_commit_id = commits_after[0].id if commits_after else None + + assert before_commit_id == after_commit_id, "Empty commit should be skipped" + + +@pytest.mark.lakefs +@pytest.mark.integration +def test_storage_options(sample_table: Table): with pytest.raises( DeltaError, match="LakeFS endpoint is missing in storage options." ): write_deltalake( "lakefs://bronze/main/oops", - data=sample_data, + data=sample_table, storage_options={ "allow_http": "true", "access_key_id": "LAKEFSID", @@ -519,7 +635,7 @@ def test_storage_options(sample_data: pa.Table): ): write_deltalake( "lakefs://bronze/main/oops", - data=sample_data, + data=sample_table, storage_options={ "endpoint": "http://127.0.0.1:8000", "allow_http": "true", diff --git a/python/tests/test_merge.py b/python/tests/test_merge.py index 7986230325..efec90900a 100644 --- a/python/tests/test_merge.py +++ b/python/tests/test_merge.py @@ -3,26 +3,34 @@ import pathlib from decimal import Decimal -import pyarrow as pa -import pyarrow.parquet as pq import pytest - -from deltalake import CommitProperties, DeltaTable, write_deltalake +from arro3.core import Array, DataType, Table +from arro3.core import Field as ArrowField + +from deltalake import ( + CommitProperties, + DeltaTable, + Field, + Schema, + write_deltalake, +) from deltalake.exceptions import DeltaProtocolError +from deltalake.query import QueryBuilder +from deltalake.schema import PrimitiveType @pytest.mark.parametrize("streaming", (True, False)) def test_merge_when_matched_delete_wo_predicate( - tmp_path: pathlib.Path, sample_table: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table: Table, streaming: bool ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["5"]), - "weight": pa.array([105], pa.int32()), + "id": Array(["5"], ArrowField("id", DataType.string(), nullable=True)), + "weight": Array([105], ArrowField("id", DataType.int32(), nullable=True)), } ) @@ -37,15 +45,34 @@ def test_merge_when_matched_delete_wo_predicate( ).when_matched_delete().execute() nrows = 4 - expected = pa.table( + + expected = Table( { - "id": pa.array(["1", "2", "3", "4"]), - "price": pa.array(list(range(nrows)), pa.int64()), - "sold": pa.array(list(range(nrows)), pa.int32()), - "deleted": pa.array([False] * nrows), - } + "id": Array( + ["1", "2", "3", "4"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + list(range(nrows)), + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + list(range(nrows)), + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False] * nrows, + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), + }, + ) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -55,20 +82,35 @@ def test_merge_when_matched_delete_wo_predicate( @pytest.mark.parametrize("streaming", (True, False)) def test_merge_when_matched_delete_with_predicate( - tmp_path: pathlib.Path, sample_table: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table: Table, streaming: bool ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["5", "4"]), - "weight": pa.array([1, 2], pa.int64()), - "sold": pa.array([1, 2], pa.int32()), - "deleted": pa.array([True, False]), - "customer": pa.array(["Adam", "Patrick"]), - } + "id": Array( + ["5", "4"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "weight": Array( + [1, 2], + ArrowField("weight", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [1, 2], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [True, False], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), + "customer": Array( + ["Adam", "Patrick"], + ArrowField("customer", type=DataType.string(), nullable=True), + ), + }, ) dt.merge( @@ -80,15 +122,33 @@ def test_merge_when_matched_delete_with_predicate( ).when_matched_delete("s.deleted = True").execute() nrows = 4 - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4"]), - "price": pa.array(list(range(nrows)), pa.int64()), - "sold": pa.array(list(range(nrows)), pa.int32()), - "deleted": pa.array([False] * nrows), - } + "id": Array( + ["1", "2", "3", "4"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + list(range(nrows)), + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + list(range(nrows)), + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False] * nrows, + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), + }, + ) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -97,18 +157,27 @@ def test_merge_when_matched_delete_with_predicate( @pytest.mark.parametrize("streaming", (True, False)) def test_merge_when_matched_update_wo_predicate( - tmp_path: pathlib.Path, sample_table: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table: Table, streaming: bool ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["4", "5"]), - "price": pa.array([10, 100], pa.int64()), - "sold": pa.array([10, 20], pa.int32()), - } + "id": Array( + ["4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + }, ) dt.merge( @@ -119,15 +188,34 @@ def test_merge_when_matched_update_wo_predicate( streamed_exec=streaming, ).when_matched_update({"price": "s.price", "sold": "s.sold"}).execute() - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5"]), - "price": pa.array([0, 1, 2, 10, 100], pa.int64()), - "sold": pa.array([0, 1, 2, 10, 20], pa.int32()), - "deleted": pa.array([False] * 5), - } + "id": Array( + ["1", "2", "3", "4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False] * 5, + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), + }, + ) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -135,19 +223,31 @@ def test_merge_when_matched_update_wo_predicate( def test_merge_when_matched_update_wo_predicate_with_schema_evolution( - tmp_path: pathlib.Path, sample_table: pa.Table + tmp_path: pathlib.Path, sample_table: Table ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["4", "5"]), - "price": pa.array([10, 100], pa.int64()), - "sold": pa.array([10, 20], pa.int32()), - "customer": pa.array(["john", "doe"]), - } + "id": Array( + ["4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "customer": Array( + ["john", "doe"], + ArrowField("customer", type=DataType.string(), nullable=True), + ), + }, ) dt.merge( @@ -160,16 +260,38 @@ def test_merge_when_matched_update_wo_predicate_with_schema_evolution( {"price": "s.price", "sold": "s.sold+int'10'", "customer": "s.customer"} ).execute() - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5"]), - "price": pa.array([0, 1, 2, 10, 100], pa.int64()), - "sold": pa.array([0, 1, 2, 20, 30], pa.int32()), - "deleted": pa.array([False] * 5), - "customer": pa.array([None, None, None, "john", "doe"]), - } + "id": Array( + ["1", "2", "3", "4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 20, 30], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False] * 5, + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), + "customer": Array( + [None, None, None, "john", "doe"], + ArrowField("customer", type=DataType.string(), nullable=True), + ), + }, + ) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -179,20 +301,35 @@ def test_merge_when_matched_update_wo_predicate_with_schema_evolution( @pytest.mark.parametrize("streaming", (True, False)) def test_merge_when_matched_update_all_wo_predicate( - tmp_path: pathlib.Path, sample_table: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table: Table, streaming: bool ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["4", "5"]), - "price": pa.array([10, 100], pa.int64()), - "sold": pa.array([10, 20], pa.int32()), - "deleted": pa.array([True, True]), - "weight": pa.array([10, 15], pa.int64()), - } + "id": Array( + ["4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [True, True], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), + "weight": Array( + [10, 15], + ArrowField("weight", type=DataType.int64(), nullable=True), + ), + }, ) dt.merge( @@ -203,15 +340,33 @@ def test_merge_when_matched_update_all_wo_predicate( streamed_exec=streaming, ).when_matched_update_all().execute() - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5"]), - "price": pa.array([0, 1, 2, 10, 100], pa.int64()), - "sold": pa.array([0, 1, 2, 10, 20], pa.int32()), - "deleted": pa.array([False, False, False, True, True]), - } + "id": Array( + ["1", "2", "3", "4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False, False, False, True, True], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), + }, + ) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -220,19 +375,34 @@ def test_merge_when_matched_update_all_wo_predicate( @pytest.mark.parametrize("streaming", (True, False)) def test_merge_when_matched_update_all_with_exclude( - tmp_path: pathlib.Path, sample_table: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table: Table, streaming: bool ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["4", "5"]), - "price": pa.array([10, 100], pa.int64()), - "sold": pa.array([15, 25], pa.int32()), - "deleted": pa.array([True, True]), - "weight": pa.array([10, 15], pa.int64()), + "id": Array( + ["4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [15, 25], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [True, True], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), + "weight": Array( + [10, 15], + ArrowField("weight", type=DataType.int64(), nullable=True), + ), } ) @@ -244,15 +414,32 @@ def test_merge_when_matched_update_all_with_exclude( streamed_exec=streaming, ).when_matched_update_all(except_cols=["sold"]).execute() - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5"]), - "price": pa.array([0, 1, 2, 10, 100], pa.int64()), - "sold": pa.array([0, 1, 2, 3, 4], pa.int32()), - "deleted": pa.array([False, False, False, True, True]), + "id": Array( + ["1", "2", "3", "4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 3, 4], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False, False, False, True, True], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -261,18 +448,30 @@ def test_merge_when_matched_update_all_with_exclude( @pytest.mark.parametrize("streaming", (True, False)) def test_merge_when_matched_update_with_predicate( - tmp_path: pathlib.Path, sample_table: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table: Table, streaming: bool ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["4", "5"]), - "price": pa.array([10, 100], pa.int64()), - "sold": pa.array([10, 20], pa.int32()), - "deleted": pa.array([False, True]), + "id": Array( + ["4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False, True], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) @@ -287,15 +486,32 @@ def test_merge_when_matched_update_with_predicate( predicate="source.deleted = False", ).execute() - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5"]), - "price": pa.array([0, 1, 2, 10, 4], pa.int64()), - "sold": pa.array([0, 1, 2, 10, 4], pa.int32()), - "deleted": pa.array([False] * 5), + "id": Array( + ["1", "2", "3", "4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 10, 4], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 10, 4], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False] * 5, + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -304,18 +520,30 @@ def test_merge_when_matched_update_with_predicate( @pytest.mark.parametrize("streaming", (True, False)) def test_merge_when_not_matched_insert_wo_predicate( - tmp_path: pathlib.Path, sample_table: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table: Table, streaming: bool ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["4", "6"]), - "price": pa.array([10, 100], pa.int64()), - "sold": pa.array([10, 20], pa.int32()), - "deleted": pa.array([False, False]), + "id": Array( + ["4", "6"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False, False], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) @@ -334,15 +562,33 @@ def test_merge_when_not_matched_insert_wo_predicate( } ).execute() - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5", "6"]), - "price": pa.array([0, 1, 2, 3, 4, 100], pa.int64()), - "sold": pa.array([0, 1, 2, 3, 4, 20], pa.int32()), - "deleted": pa.array([False] * 6), + "id": Array( + ["1", "2", "3", "4", "5", "6"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 3, 4, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 3, 4, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False] * 6, + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -351,18 +597,30 @@ def test_merge_when_not_matched_insert_wo_predicate( @pytest.mark.parametrize("streaming", (True, False)) def test_merge_when_not_matched_insert_with_predicate( - tmp_path: pathlib.Path, sample_table: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table: Table, streaming: bool ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["6", "10"]), - "price": pa.array([10, 100], pa.int64()), - "sold": pa.array([10, 20], pa.int32()), - "deleted": pa.array([False, False]), + "id": Array( + ["6", "10"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False, False], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) @@ -382,15 +640,33 @@ def test_merge_when_not_matched_insert_with_predicate( predicate="source.price < 50", ).execute() - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5", "6"]), - "price": pa.array([0, 1, 2, 3, 4, 10], pa.int64()), - "sold": pa.array([0, 1, 2, 3, 4, 10], pa.int32()), - "deleted": pa.array([False] * 6), + "id": Array( + ["1", "2", "3", "4", "5", "6"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 3, 4, 10], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 3, 4, 10], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False] * 6, + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -398,19 +674,34 @@ def test_merge_when_not_matched_insert_with_predicate( def test_merge_when_not_matched_insert_with_predicate_schema_evolution( - tmp_path: pathlib.Path, sample_table: pa.Table + tmp_path: pathlib.Path, sample_table: Table ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["6", "10"]), - "price": pa.array([10, 100], pa.int64()), - "sold": pa.array([10, 20], pa.int32()), - "customer": pa.array(["john", "doe"]), - "deleted": pa.array([False, False]), + "id": Array( + ["6", "10"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "customer": Array( + ["john", "doe"], + ArrowField("customer", type=DataType.string(), nullable=True), + ), + "deleted": Array( + [False, False], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) @@ -431,16 +722,37 @@ def test_merge_when_not_matched_insert_with_predicate_schema_evolution( predicate="source.price < 50", ).execute() - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5", "6"]), - "price": pa.array([0, 1, 2, 3, 4, 10], pa.int64()), - "sold": pa.array([0, 1, 2, 3, 4, 10], pa.int32()), - "deleted": pa.array([False] * 6), - "customer": pa.array([None, None, None, None, None, "john"]), + "id": Array( + ["1", "2", "3", "4", "5", "6"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 3, 4, 10], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 3, 4, 10], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False] * 6, + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), + "customer": Array( + [None, None, None, None, None, "john"], + ArrowField("customer", type=DataType.string(), nullable=True), + ), } ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -450,18 +762,30 @@ def test_merge_when_not_matched_insert_with_predicate_schema_evolution( @pytest.mark.parametrize("streaming", (True, False)) def test_merge_when_not_matched_insert_all_with_predicate( - tmp_path: pathlib.Path, sample_table: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table: Table, streaming: bool ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["6", "10"]), - "price": pa.array([10, 100], pa.int64()), - "sold": pa.array([10, 20], pa.int32()), - "deleted": pa.array([None, None], pa.bool_()), + "id": Array( + ["6", "10"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [None, None], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) @@ -475,15 +799,33 @@ def test_merge_when_not_matched_insert_all_with_predicate( predicate="source.price < 50", ).execute() - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5", "6"]), - "price": pa.array([0, 1, 2, 3, 4, 10], pa.int64()), - "sold": pa.array([0, 1, 2, 3, 4, 10], pa.int32()), - "deleted": pa.array([False, False, False, False, False, None]), + "id": Array( + ["1", "2", "3", "4", "5", "6"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 3, 4, 10], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 3, 4, 10], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False, False, False, False, False, None], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -492,18 +834,30 @@ def test_merge_when_not_matched_insert_all_with_predicate( @pytest.mark.parametrize("streaming", (True, False)) def test_merge_when_not_matched_insert_all_with_exclude( - tmp_path: pathlib.Path, sample_table: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table: Table, streaming: bool ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["6", "9"]), - "price": pa.array([10, 100], pa.int64()), - "sold": pa.array([10, 20], pa.int32()), - "deleted": pa.array([None, None], pa.bool_()), + "id": Array( + ["6", "9"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [None, None], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) @@ -515,15 +869,33 @@ def test_merge_when_not_matched_insert_all_with_exclude( streamed_exec=streaming, ).when_not_matched_insert_all(except_cols=["sold"]).execute() - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5", "6", "9"]), - "price": pa.array([0, 1, 2, 3, 4, 10, 100], pa.int64()), - "sold": pa.array([0, 1, 2, 3, 4, None, None], pa.int32()), - "deleted": pa.array([False, False, False, False, False, None, None]), + "id": Array( + ["1", "2", "3", "4", "5", "6", "9"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 3, 4, 10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 3, 4, None, None], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False, False, False, False, False, None, None], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -531,19 +903,34 @@ def test_merge_when_not_matched_insert_all_with_exclude( def test_merge_when_not_matched_insert_all_with_exclude_and_with_schema_evo( - tmp_path: pathlib.Path, sample_table: pa.Table + tmp_path: pathlib.Path, sample_table: Table ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["6", "9"]), - "price": pa.array([10, 100], pa.int64()), - "sold": pa.array([10, 20], pa.int32()), - "deleted": pa.array([None, None], pa.bool_()), - "customer": pa.array(["john", "doe"]), + "id": Array( + ["6", "9"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [None, None], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), + "customer": Array( + ["john", "doe"], + ArrowField("customer", type=DataType.string(), nullable=True), + ), } ) @@ -555,16 +942,37 @@ def test_merge_when_not_matched_insert_all_with_exclude_and_with_schema_evo( predicate="target.id = source.id", ).when_not_matched_insert_all(except_cols=["sold"]).execute() - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5", "6", "9"]), - "price": pa.array([0, 1, 2, 3, 4, 10, 100], pa.int64()), - "sold": pa.array([0, 1, 2, 3, 4, None, None], pa.int32()), - "deleted": pa.array([False, False, False, False, False, None, None]), - "customer": pa.array([None, None, None, None, None, "john", "doe"]), + "id": Array( + ["1", "2", "3", "4", "5", "6", "9"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 3, 4, 10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 3, 4, None, None], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False, False, False, False, False, None, None], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), + "customer": Array( + [None, None, None, None, None, "john", "doe"], + ArrowField("customer", type=DataType.string(), nullable=True), + ), } ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -574,18 +982,30 @@ def test_merge_when_not_matched_insert_all_with_exclude_and_with_schema_evo( @pytest.mark.parametrize("streaming", (True, False)) def test_merge_when_not_matched_insert_all_with_predicate_special_column_names( - tmp_path: pathlib.Path, sample_table_with_spaces_numbers: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table_with_spaces_numbers: Table, streaming: bool ): write_deltalake(tmp_path, sample_table_with_spaces_numbers, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "1id": pa.array(["6", "10"]), - "price": pa.array([10, 100], pa.int64()), - "sold items": pa.array([10, 20], pa.int32()), - "deleted": pa.array([None, None], pa.bool_()), + "1id": Array( + ["6", "10"], + ArrowField("1id", type=DataType.string(), nullable=True), + ), + "price": Array( + [10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold items": Array( + [10, 20], + ArrowField("sold items", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [None, None], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) @@ -599,15 +1019,33 @@ def test_merge_when_not_matched_insert_all_with_predicate_special_column_names( predicate="source.price < 50", ).execute() - expected = pa.table( + expected = Table( { - "1id": pa.array(["1", "2", "3", "4", "5", "6"]), - "price": pa.array([0, 1, 2, 3, 4, 10], pa.int64()), - "sold items": pa.array([0, 1, 2, 3, 4, 10], pa.int32()), - "deleted": pa.array([False, False, False, False, False, None]), + "1id": Array( + ["1", "2", "3", "4", "5", "6"], + ArrowField("1id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 3, 4, 10], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold items": Array( + [0, 1, 2, 3, 4, 10], + ArrowField("sold items", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False, False, False, False, False, None], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) - result = dt.to_pyarrow_table().sort_by([("1id", "ascending")]) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by `1id` asc") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -616,18 +1054,30 @@ def test_merge_when_not_matched_insert_all_with_predicate_special_column_names( @pytest.mark.parametrize("streaming", (True, False)) def test_merge_when_not_matched_by_source_update_wo_predicate( - tmp_path: pathlib.Path, sample_table: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table: Table, streaming: bool ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["6", "7"]), - "price": pa.array([10, 100], pa.int64()), - "sold": pa.array([10, 20], pa.int32()), - "deleted": pa.array([False, False]), + "id": Array( + ["6", "7"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False, False], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) @@ -643,15 +1093,33 @@ def test_merge_when_not_matched_by_source_update_wo_predicate( } ).execute() - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5"]), - "price": pa.array([0, 1, 2, 3, 4], pa.int64()), - "sold": pa.array([10, 10, 10, 10, 10], pa.int32()), - "deleted": pa.array([False] * 5), + "id": Array( + ["1", "2", "3", "4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 3, 4], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 10, 10, 10, 10], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False] * 5, + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -660,18 +1128,30 @@ def test_merge_when_not_matched_by_source_update_wo_predicate( @pytest.mark.parametrize("streaming", (True, False)) def test_merge_when_not_matched_by_source_update_with_predicate( - tmp_path: pathlib.Path, sample_table: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table: Table, streaming: bool ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["6", "7"]), - "price": pa.array([10, 100], pa.int64()), - "sold": pa.array([10, 20], pa.int32()), - "deleted": pa.array([False, False]), + "id": Array( + ["6", "7"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False, False], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) @@ -688,15 +1168,33 @@ def test_merge_when_not_matched_by_source_update_with_predicate( predicate="target.price > 3", ).execute() - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5"]), - "price": pa.array([0, 1, 2, 3, 4], pa.int64()), - "sold": pa.array([0, 1, 2, 3, 10], pa.int32()), - "deleted": pa.array([False] * 5), + "id": Array( + ["1", "2", "3", "4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 3, 4], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 3, 10], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False] * 5, + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -705,20 +1203,33 @@ def test_merge_when_not_matched_by_source_update_with_predicate( @pytest.mark.parametrize("streaming", (True, False)) def test_merge_when_not_matched_by_source_delete_with_predicate( - tmp_path: pathlib.Path, sample_table: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table: Table, streaming: bool ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["6", "7"]), - "price": pa.array([10, 100], pa.int64()), - "sold": pa.array([10, 20], pa.int32()), - "deleted": pa.array([False, False]), + "id": Array( + ["6", "7"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False, False], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) + dt.merge( source=source_table, source_alias="source", @@ -727,18 +1238,33 @@ def test_merge_when_not_matched_by_source_delete_with_predicate( streamed_exec=streaming, ).when_not_matched_by_source_delete(predicate="target.price > 3").execute() - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4"]), - "price": pa.array( + "id": Array( + ["1", "2", "3", "4"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( [0, 1, 2, 3], - pa.int64(), + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 3], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False] * 4, + ArrowField("deleted", type=DataType.bool(), nullable=True), ), - "sold": pa.array([0, 1, 2, 3], pa.int32()), - "deleted": pa.array([False] * 4), } ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -747,14 +1273,23 @@ def test_merge_when_not_matched_by_source_delete_with_predicate( @pytest.mark.parametrize("streaming", (True, False)) def test_merge_when_not_matched_by_source_delete_wo_predicate( - tmp_path: pathlib.Path, sample_table: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table: Table, streaming: bool ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( - {"id": pa.array(["4", "5"]), "weight": pa.array([1.5, 1.6], pa.float64())} + source_table = Table( + { + "id": Array( + ["4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "weight": Array( + [1.5, 1.6], + ArrowField("weight", type=DataType.float64(), nullable=True), + ), + } ) dt.merge( @@ -765,18 +1300,33 @@ def test_merge_when_not_matched_by_source_delete_wo_predicate( streamed_exec=streaming, ).when_not_matched_by_source_delete().execute() - expected = pa.table( + expected = Table( { - "id": pa.array(["4", "5"]), - "price": pa.array( + "id": Array( + ["4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( [3, 4], - pa.int64(), + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [3, 4], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False] * 2, + ArrowField("deleted", type=DataType.bool(), nullable=True), ), - "sold": pa.array([3, 4], pa.int32()), - "deleted": pa.array([False] * 2), } ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -785,18 +1335,30 @@ def test_merge_when_not_matched_by_source_delete_wo_predicate( @pytest.mark.parametrize("streaming", (True, False)) def test_merge_multiple_when_matched_update_with_predicate( - tmp_path: pathlib.Path, sample_table: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table: Table, streaming: bool ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["4", "5"]), - "price": pa.array([10, 100], pa.int64()), - "sold": pa.array([10, 20], pa.int32()), - "deleted": pa.array([False, True]), + "id": Array( + ["4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False, True], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) @@ -814,15 +1376,33 @@ def test_merge_multiple_when_matched_update_with_predicate( predicate="source.deleted = True", ).execute() - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5"]), - "price": pa.array([0, 1, 2, 10, 100], pa.int64()), - "sold": pa.array([0, 1, 2, 10, 20], pa.int32()), - "deleted": pa.array([False] * 5), + "id": Array( + ["1", "2", "3", "4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False] * 5, + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -831,18 +1411,30 @@ def test_merge_multiple_when_matched_update_with_predicate( @pytest.mark.parametrize("streaming", (True, False)) def test_merge_multiple_when_matched_update_all_with_predicate( - tmp_path: pathlib.Path, sample_table: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table: Table, streaming: bool ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["4", "5"]), - "price": pa.array([10, 100], pa.int64()), - "sold": pa.array([10, 20], pa.int32()), - "deleted": pa.array([False, True]), + "id": Array( + ["4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False, True], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) @@ -858,15 +1450,33 @@ def test_merge_multiple_when_matched_update_all_with_predicate( predicate="source.deleted = True", ).execute() - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5"]), - "price": pa.array([0, 1, 2, 10, 100], pa.int64()), - "sold": pa.array([0, 1, 2, 10, 20], pa.int32()), - "deleted": pa.array([False, False, False, False, True]), + "id": Array( + ["1", "2", "3", "4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False, False, False, False, True], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -875,18 +1485,30 @@ def test_merge_multiple_when_matched_update_all_with_predicate( @pytest.mark.parametrize("streaming", (True, False)) def test_merge_multiple_when_not_matched_insert_with_predicate( - tmp_path: pathlib.Path, sample_table: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table: Table, streaming: bool ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["6", "9"]), - "price": pa.array([10, 100], pa.int64()), - "sold": pa.array([10, 20], pa.int32()), - "deleted": pa.array([False, False]), + "id": Array( + ["6", "9"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False, False], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) @@ -914,15 +1536,33 @@ def test_merge_multiple_when_not_matched_insert_with_predicate( predicate="source.price > 50", ).execute() - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5", "6", "9"]), - "price": pa.array([0, 1, 2, 3, 4, 10, 100], pa.int64()), - "sold": pa.array([0, 1, 2, 3, 4, 10, 20], pa.int32()), - "deleted": pa.array([False] * 7), + "id": Array( + ["1", "2", "3", "4", "5", "6", "9"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 3, 4, 10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 3, 4, 10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False] * 7, + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -931,19 +1571,34 @@ def test_merge_multiple_when_not_matched_insert_with_predicate( @pytest.mark.parametrize("streaming", (True, False)) def test_merge_multiple_when_matched_delete_with_predicate( - tmp_path: pathlib.Path, sample_table: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table: Table, streaming: bool ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["5", "4"]), - "weight": pa.array([1, 2], pa.int64()), - "sold": pa.array([1, 2], pa.int32()), - "deleted": pa.array([True, False]), - "customer": pa.array(["Adam", "Patrick"]), + "id": Array( + ["5", "4"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "weight": Array( + [1, 2], + ArrowField("weight", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [1, 2], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [True, False], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), + "customer": Array( + ["Adam", "Patrick"], + ArrowField("customer", type=DataType.string(), nullable=True), + ), } ) @@ -957,16 +1612,33 @@ def test_merge_multiple_when_matched_delete_with_predicate( "s.deleted = false" ).execute() - nrows = 3 - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3"]), - "price": pa.array(list(range(nrows)), pa.int64()), - "sold": pa.array(list(range(nrows)), pa.int32()), - "deleted": pa.array([False] * nrows), + "id": Array( + ["1", "2", "3"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + list(range(3)), + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + list(range(3)), + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False] * 3, + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -975,7 +1647,7 @@ def test_merge_multiple_when_matched_delete_with_predicate( @pytest.mark.parametrize("streaming", (True, False)) def test_merge_multiple_when_not_matched_by_source_update_wo_predicate( - tmp_path: pathlib.Path, sample_table: pa.Table, streaming: bool + tmp_path: pathlib.Path, sample_table: Table, streaming: bool ): """The first match clause that meets the predicate will be executed, so if you do an update without an other predicate the first clause will be matched and therefore the other ones are skipped. @@ -984,12 +1656,24 @@ def test_merge_multiple_when_not_matched_by_source_update_wo_predicate( dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["6", "7"]), - "price": pa.array([10, 100], pa.int64()), - "sold": pa.array([10, 20], pa.int32()), - "deleted": pa.array([False, False]), + "id": Array( + ["6", "7"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [10, 100], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False, False], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) @@ -1009,30 +1693,51 @@ def test_merge_multiple_when_not_matched_by_source_update_wo_predicate( } ).execute() - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5"]), - "price": pa.array([0, 1, 2, 3, 4], pa.int64()), - "sold": pa.array([10, 10, 10, 10, 10], pa.int32()), - "deleted": pa.array([False] * 5), + "id": Array( + ["1", "2", "3", "4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 3, 4], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 10, 10, 10, 10], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), + "deleted": Array( + [False] * 5, + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id asc") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" assert result == expected +@pytest.mark.pyarrow @pytest.mark.parametrize("streaming", (True, False)) def test_merge_date_partitioned_2344(tmp_path: pathlib.Path, streaming: bool): from datetime import date - schema = pa.schema( - [ - ("date", pa.date32()), - ("foo", pa.string()), - ("bar", pa.string()), + import pyarrow as pa + + schema = Schema( + fields=[ + Field("date", PrimitiveType("date")), + Field("foo", PrimitiveType("string")), + Field("bar", PrimitiveType("string")), ] ) @@ -1072,6 +1777,7 @@ def test_merge_date_partitioned_2344(tmp_path: pathlib.Path, streaming: bool): assert last_action["operationParameters"].get("predicate") is None +@pytest.mark.pyarrow @pytest.mark.parametrize( "timezone,predicate", [ @@ -1088,11 +1794,18 @@ def test_merge_date_partitioned_2344(tmp_path: pathlib.Path, streaming: bool): def test_merge_timestamps_partitioned_2344(tmp_path: pathlib.Path, timezone, predicate): from datetime import datetime - schema = pa.schema( - [ - ("datetime", pa.timestamp("us", tz=timezone)), - ("foo", pa.string()), - ("bar", pa.string()), + import pyarrow as pa + + schema = Schema( + fields=[ + Field( + "datetime", + PrimitiveType("timestamp") + if timezone + else PrimitiveType("timestamp_ntz"), + ), + Field("foo", PrimitiveType("string")), + Field("bar", PrimitiveType("string")), ] ) @@ -1128,13 +1841,23 @@ def test_merge_timestamps_partitioned_2344(tmp_path: pathlib.Path, timezone, pre @pytest.mark.parametrize("streaming", (True, False)) def test_merge_stats_columns_stats_provided(tmp_path: pathlib.Path, streaming: bool): - data = pa.table( + data = Table( { - "foo": pa.array(["a", "b", None, None]), - "bar": pa.array([1, 2, 3, None]), - "baz": pa.array([1, 1, None, None]), + "foo": Array( + ["a", "b", None, None], + ArrowField("foo", type=DataType.string(), nullable=True), + ), + "bar": Array( + [1, 2, 3, None], + ArrowField("bar", type=DataType.int64(), nullable=True), + ), + "baz": Array( + [1, 1, None, None], + ArrowField("baz", type=DataType.int64(), nullable=True), + ), } ) + write_deltalake( tmp_path, data, @@ -1143,23 +1866,35 @@ def test_merge_stats_columns_stats_provided(tmp_path: pathlib.Path, streaming: b ) dt = DeltaTable(tmp_path) add_actions_table = dt.get_add_actions(flatten=True) - stats = add_actions_table.to_pylist()[0] - - assert stats["null_count.foo"] == 2 - assert stats["min.foo"] == "a" - assert stats["max.foo"] == "b" - assert stats["null_count.bar"] is None - assert stats["min.bar"] is None - assert stats["max.bar"] is None - assert stats["null_count.baz"] == 2 - assert stats["min.baz"] == 1 - assert stats["max.baz"] == 1 - data = pa.table( + def get_value(name: str): + return add_actions_table.column(name)[0].as_py() + + # x1 has no max, since inf was the highest value + assert get_value("null_count.foo") == 2 + assert get_value("min.foo") == "a" + assert get_value("max.foo") == "b" + assert get_value("null_count.bar") is None + assert get_value("min.bar") is None + assert get_value("max.bar") is None + assert get_value("null_count.baz") == 2 + assert get_value("min.baz") == 1 + assert get_value("max.baz") == 1 + + data = Table( { - "foo": pa.array(["a"]), - "bar": pa.array([10]), - "baz": pa.array([10]), + "foo": Array( + ["a"], + ArrowField("foo", type=DataType.string(), nullable=True), + ), + "bar": Array( + [10], + ArrowField("bar", type=DataType.int64(), nullable=True), + ), + "baz": Array( + [10], + ArrowField("baz", type=DataType.int64(), nullable=True), + ), } ) @@ -1173,27 +1908,50 @@ def test_merge_stats_columns_stats_provided(tmp_path: pathlib.Path, streaming: b dt = DeltaTable(tmp_path) add_actions_table = dt.get_add_actions(flatten=True) - stats = add_actions_table.to_pylist()[0] assert dt.version() == 1 - assert stats["null_count.foo"] == 2 - assert stats["min.foo"] == "a" - assert stats["max.foo"] == "b" - assert stats["null_count.bar"] is None - assert stats["min.bar"] is None - assert stats["max.bar"] is None - assert stats["null_count.baz"] == 2 - assert stats["min.baz"] == 1 - assert stats["max.baz"] == 10 + + def get_value(name: str): + return add_actions_table.column(name)[0].as_py() + + # x1 has no max, since inf was the highest value + assert get_value("null_count.foo") == 2 + assert get_value("min.foo") == "a" + assert get_value("max.foo") == "b" + assert get_value("null_count.bar") is None + assert get_value("min.bar") is None + assert get_value("max.bar") is None + assert get_value("null_count.baz") == 2 + assert get_value("min.baz") == 1 + assert get_value("max.baz") == 10 def test_merge_field_special_characters_delete_2438(tmp_path: pathlib.Path): ## See issue: https://github.com/delta-io/delta-rs/issues/2438 - data = pa.table({"x": [1, 2, 3], "y--1": [4, 5, 6]}) + data = Table( + { + "x": Array( + [1, 2, 3], + ArrowField("x", type=DataType.int64(), nullable=True), + ), + "y--1": Array( + [4, 5, 6], + ArrowField("y--1", type=DataType.int64(), nullable=True), + ), + } + ) + write_deltalake(tmp_path, data, mode="append") dt = DeltaTable(tmp_path) - new_data = pa.table({"x": [2, 3]}) + new_data = Table( + { + "x": Array( + [2, 3], + ArrowField("x", type=DataType.int64(), nullable=True), + ), + } + ) ( dt.merge( @@ -1206,22 +1964,36 @@ def test_merge_field_special_characters_delete_2438(tmp_path: pathlib.Path): .execute() ) - expected = pa.table({"x": [1], "y--1": [4]}) + expected = Table( + { + "x": Array( + [1], + ArrowField("x", type=DataType.int64(), nullable=True), + ), + "y--1": Array( + [4], + ArrowField("y--1", type=DataType.int64(), nullable=True), + ), + } + ) - assert dt.to_pyarrow_table() == expected + assert ( + QueryBuilder().register("tbl", dt).execute("select * from tbl").read_all() + ) == expected @pytest.mark.pandas +@pytest.mark.pyarrow def test_struct_casting(tmp_path: pathlib.Path): import pandas as pd + import pyarrow as pa - cols = ["id", "name", "address", "scores"] + cols = ["id", "name", "address"] data = [ ( 2, "Marry Doe", {"street": "123 Main St", "city": "Anytown", "state": "CA"}, - [0, 0, 0], ) ] df = pd.DataFrame(data, columns=cols) @@ -1231,7 +2003,6 @@ def test_struct_casting(tmp_path: pathlib.Path): 2, "Merged", {"street": "1 Front", "city": "San Francisco", "state": "CA"}, - [7, 0, 7], ) ], columns=cols, @@ -1239,7 +2010,8 @@ def test_struct_casting(tmp_path: pathlib.Path): assert not df.empty schema = pa.Table.from_pandas(df=df).schema - dt = DeltaTable.create(tmp_path, schema, name="test") + + dt = DeltaTable.create(tmp_path, Schema.from_arrow(schema), name="test") metadata = dt.metadata() assert metadata.name == "test" @@ -1259,11 +2031,20 @@ def test_struct_casting(tmp_path: pathlib.Path): @pytest.mark.parametrize("streaming", (True, False)) def test_merge_isin_partition_pruning(tmp_path: pathlib.Path, streaming: bool): nrows = 5 - data = pa.table( + data = Table( { - "id": pa.array([str(x) for x in range(nrows)]), - "partition": pa.array(list(range(nrows)), pa.int64()), - "sold": pa.array(list(range(nrows)), pa.int32()), + "id": Array( + [str(x) for x in range(nrows)], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "partition": Array( + list(range(nrows)), + ArrowField("partition", type=DataType.int64(), nullable=True), + ), + "sold": Array( + list(range(nrows)), + ArrowField("sold", type=DataType.int32(), nullable=True), + ), } ) @@ -1271,11 +2052,20 @@ def test_merge_isin_partition_pruning(tmp_path: pathlib.Path, streaming: bool): dt = DeltaTable(tmp_path) - source_table = pa.table( + source_table = Table( { - "id": pa.array(["3", "4"]), - "partition": pa.array([3, 4], pa.int64()), - "sold": pa.array([10, 20], pa.int32()), + "id": Array( + ["3", "4"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "partition": Array( + [3, 4], + ArrowField("partition", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), } ) @@ -1291,14 +2081,29 @@ def test_merge_isin_partition_pruning(tmp_path: pathlib.Path, streaming: bool): .execute() ) - expected = pa.table( + expected = Table( { - "id": pa.array(["0", "1", "2", "3", "4"]), - "partition": pa.array([0, 1, 2, 3, 4], pa.int64()), - "sold": pa.array([0, 1, 2, 10, 20], pa.int32()), + "id": Array( + ["0", "1", "2", "3", "4"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "partition": Array( + [0, 1, 2, 3, 4], + ArrowField("partition", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 10, 20], + ArrowField("sold", type=DataType.int32(), nullable=True), + ), } ) - result = dt.to_pyarrow_table().sort_by([("id", "ascending")]) + + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select id, partition, sold from tbl order by id asc") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "MERGE" @@ -1307,10 +2112,12 @@ def test_merge_isin_partition_pruning(tmp_path: pathlib.Path, streaming: bool): assert metrics["num_target_files_skipped_during_scan"] == 3 +@pytest.mark.pyarrow @pytest.mark.parametrize("streaming", (True, False)) def test_cdc_merge_planning_union_2908(tmp_path, streaming: bool): """https://github.com/delta-io/delta-rs/issues/2908""" cdc_path = f"{tmp_path}/_change_data" + import pyarrow as pa data = { "id": pa.array([1, 2], pa.int64()), @@ -1323,7 +2130,7 @@ def test_cdc_merge_planning_union_2908(tmp_path, streaming: bool): dt = DeltaTable.create( table_uri=tmp_path, - schema=table.schema, + schema=Schema.from_arrow(table.schema), mode="overwrite", partition_by=["id"], configuration={ @@ -1347,6 +2154,7 @@ def test_cdc_merge_planning_union_2908(tmp_path, streaming: bool): @pytest.mark.pandas +@pytest.mark.pyarrow def test_merge_non_nullable(tmp_path): import re @@ -1385,9 +2193,13 @@ def test_merge_non_nullable(tmp_path): ).when_matched_update_all().when_not_matched_insert_all().execute() +@pytest.mark.pyarrow def test_merge_when_wrong_but_castable_type_passed_while_merge( - tmp_path: pathlib.Path, sample_table: pa.Table + tmp_path: pathlib.Path, sample_table: Table ): + import pyarrow as pa + import pyarrow.parquet as pq + write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) @@ -1413,7 +2225,10 @@ def test_merge_when_wrong_but_castable_type_passed_while_merge( assert table_schema.field("price").type == sample_table["price"].type +@pytest.mark.pyarrow def test_merge_on_decimal_3033(tmp_path): + import pyarrow as pa + data = { "timestamp": [datetime.datetime(2024, 3, 20, 12, 30, 0)], "altitude": [Decimal("150.5")], @@ -1421,11 +2236,13 @@ def test_merge_on_decimal_3033(tmp_path): table = pa.Table.from_pydict(data) - schema = pa.schema( - [ - ("timestamp", pa.timestamp("us")), - ("altitude", pa.decimal128(6, 1)), - ] + schema = Schema.from_arrow( + pa.schema( + [ + ("timestamp", pa.timestamp("us")), + ("altitude", pa.decimal128(6, 1)), + ] + ) ) dt = DeltaTable.create(tmp_path, schema=schema) @@ -1452,11 +2269,12 @@ def test_merge_on_decimal_3033(tmp_path): assert ( string_predicate - == "timestamp >= arrow_cast('2024-03-20T12:30:00.000000', 'Timestamp(Microsecond, None)') AND timestamp <= arrow_cast('2024-03-20T12:30:00.000000', 'Timestamp(Microsecond, None)') AND altitude >= '1505'::decimal(4, 1) AND altitude <= '1505'::decimal(4, 1)" + == "timestamp = arrow_cast('2024-03-20T12:30:00.000000', 'Timestamp(Microsecond, None)') AND altitude = '1505'::decimal(4, 1)" ) @pytest.mark.polars +@pytest.mark.xfail(reason="polars will require an update") def test_merge(tmp_path: pathlib.Path): import polars as pl from polars.testing import assert_frame_equal diff --git a/python/tests/test_nopyarrow.py b/python/tests/test_nopyarrow.py new file mode 100644 index 0000000000..f1f0bd7784 --- /dev/null +++ b/python/tests/test_nopyarrow.py @@ -0,0 +1,31 @@ +import pytest + +from deltalake import DeltaTable + + +@pytest.mark.no_pyarrow +def test_import_error_reading_pyarrow(existing_sample_table: DeltaTable): + with pytest.raises(ImportError, match="pyarrow"): + existing_sample_table.to_pyarrow_dataset() + + with pytest.raises(ImportError, match="pyarrow"): + existing_sample_table.to_pyarrow_table() + + with pytest.raises(ImportError, match="pyarrow"): + existing_sample_table.to_pandas() + + +@pytest.mark.no_pyarrow +def test_import_error_storage_handler_from_table(existing_sample_table: DeltaTable): + from deltalake.fs import DeltaStorageHandler + + with pytest.raises(ImportError, match="pyarrow"): + DeltaStorageHandler.from_table(existing_sample_table) + + +@pytest.mark.no_pyarrow +def test_import_error_storage_handler_init(): + from deltalake.fs import DeltaStorageHandler + + with pytest.raises(ImportError, match="pyarrow"): + DeltaStorageHandler(table_uri="random") diff --git a/python/tests/test_optimize.py b/python/tests/test_optimize.py index 899432e874..bfe8842b34 100644 --- a/python/tests/test_optimize.py +++ b/python/tests/test_optimize.py @@ -1,23 +1,18 @@ import pathlib from datetime import timedelta -import pyarrow as pa import pytest - -try: - import pandas as pd -except ModuleNotFoundError: - _has_pandas = False -else: - _has_pandas = True +from arro3.core import Array, DataType, Table +from arro3.core import Field as ArrowField from deltalake import CommitProperties, DeltaTable, write_deltalake +from deltalake.query import QueryBuilder @pytest.mark.parametrize("use_relative", [True, False]) def test_optimize_run_table( tmp_path: pathlib.Path, - sample_data: pa.Table, + sample_table: Table, monkeypatch, use_relative: bool, ): @@ -28,18 +23,28 @@ def test_optimize_run_table( else: table_path = str(tmp_path) - write_deltalake(table_path, sample_data, mode="append") - write_deltalake(table_path, sample_data, mode="append") - write_deltalake(table_path, sample_data, mode="append") + write_deltalake(table_path, sample_table, mode="append") + write_deltalake(table_path, sample_table, mode="append") + write_deltalake(table_path, sample_table, mode="append") dt = DeltaTable(table_path) - old_data = dt.to_pyarrow_table() + old_data = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id") + .read_all() + ) old_version = dt.version() commit_properties = CommitProperties(custom_metadata={"userName": "John Doe"}) dt.optimize.compact(commit_properties=commit_properties) - new_data = dt.to_pyarrow_table() + new_data = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by id") + .read_all() + ) last_action = dt.history(1)[0] assert last_action["operation"] == "OPTIMIZE" assert last_action["userName"] == "John Doe" @@ -49,21 +54,21 @@ def test_optimize_run_table( def test_z_order_optimize( tmp_path: pathlib.Path, - sample_data: pa.Table, + sample_table: Table, ): write_deltalake( tmp_path, - sample_data, + sample_table, mode="append", ) write_deltalake( tmp_path, - sample_data, + sample_table, mode="append", ) write_deltalake( tmp_path, - sample_data, + sample_table, mode="append", ) @@ -71,7 +76,7 @@ def test_z_order_optimize( old_version = dt.version() commit_properties = CommitProperties(custom_metadata={"userName": "John Doe"}) - dt.optimize.z_order(["date32", "timestamp"], commit_properties=commit_properties) + dt.optimize.z_order(["sold", "price"], commit_properties=commit_properties) last_action = dt.history(1)[0] assert last_action["operation"] == "OPTIMIZE" assert last_action["userName"] == "John Doe" @@ -81,16 +86,16 @@ def test_z_order_optimize( def test_optimize_min_commit_interval( tmp_path: pathlib.Path, - sample_data: pa.Table, + sample_table: Table, ): - write_deltalake(tmp_path, sample_data, partition_by="utf8", mode="append") - write_deltalake(tmp_path, sample_data, partition_by="utf8", mode="append") - write_deltalake(tmp_path, sample_data, partition_by="utf8", mode="append") + write_deltalake(tmp_path, sample_table, partition_by="id", mode="append") + write_deltalake(tmp_path, sample_table, partition_by="id", mode="append") + write_deltalake(tmp_path, sample_table, partition_by="id", mode="append") dt = DeltaTable(tmp_path) old_version = dt.version() - dt.optimize.z_order(["date32", "timestamp"], min_commit_interval=timedelta(0)) + dt.optimize.z_order(["sold", "price"], min_commit_interval=timedelta(0)) last_action = dt.history(1)[0] assert last_action["operation"] == "OPTIMIZE" @@ -102,13 +107,28 @@ def test_optimize_min_commit_interval( def test_optimize_schema_evolved_table( tmp_path: pathlib.Path, - sample_data: pa.Table, + sample_table: Table, ): - data = pa.table({"foo": pa.array(["1"])}) + data = Table( + { + "foo": Array( + ["1"], + ArrowField("foo", type=DataType.string(), nullable=True), + ), + } + ) write_deltalake(tmp_path, data, mode="append", schema_mode="merge") - data = pa.table({"bar": pa.array(["1"])}) + data = Table( + { + "bar": Array( + ["1"], + ArrowField("bar", type=DataType.string(), nullable=True), + ), + } + ) + write_deltalake(tmp_path, data, mode="append", schema_mode="merge") dt = DeltaTable(tmp_path) @@ -120,20 +140,33 @@ def test_optimize_schema_evolved_table( assert last_action["operation"] == "OPTIMIZE" assert dt.version() == old_version + 1 - data = pa.table( + data = Table( { - "foo": pa.array([None, "1"]), - "bar": pa.array(["1", None]), + "foo": Array( + ["1", None], + ArrowField("foo", type=DataType.string(), nullable=True), + ), + "bar": Array( + [None, "1"], + ArrowField("bar", type=DataType.string(), nullable=True), + ), } ) - assert dt.to_pyarrow_table().sort_by([("foo", "ascending")]) == data.sort_by( - [("foo", "ascending")] + assert ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl order by foo asc") + .read_all() + == data ) @pytest.mark.pandas +@pytest.mark.pyarrow def test_zorder_with_space_partition(tmp_path: pathlib.Path): + import pandas as pd + df = pd.DataFrame( { "user": ["James", "Anna", "Sara", "Martin"], @@ -160,8 +193,10 @@ def test_zorder_with_space_partition(tmp_path: pathlib.Path): test_table.optimize.z_order(columns=["user"]) +@pytest.mark.pyarrow def test_optimize_schema_evolved_3185(tmp_path): """https://github.com/delta-io/delta-rs/issues/3185""" + import pyarrow as pa # Define the data for the first write data_first_write = pa.array( diff --git a/python/tests/test_repair.py b/python/tests/test_repair.py index 52ef7e8b07..ede399d992 100644 --- a/python/tests/test_repair.py +++ b/python/tests/test_repair.py @@ -3,9 +3,9 @@ from deltalake import CommitProperties, DeltaTable, write_deltalake -def test_repair_with_dry_run(tmp_path, sample_data): - write_deltalake(tmp_path, sample_data, mode="append") - write_deltalake(tmp_path, sample_data, mode="append") +def test_repair_with_dry_run(tmp_path, sample_table): + write_deltalake(tmp_path, sample_table, mode="append") + write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) os.remove(dt.file_uris()[0]) @@ -17,9 +17,9 @@ def test_repair_with_dry_run(tmp_path, sample_data): assert last_action["operation"] == "WRITE" -def test_repair_wo_dry_run(tmp_path, sample_data): - write_deltalake(tmp_path, sample_data, mode="append") - write_deltalake(tmp_path, sample_data, mode="append") +def test_repair_wo_dry_run(tmp_path, sample_table): + write_deltalake(tmp_path, sample_table, mode="append") + write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) os.remove(dt.file_uris()[0]) diff --git a/python/tests/test_restore.py b/python/tests/test_restore.py index a38515c794..fd9299a3ad 100644 --- a/python/tests/test_restore.py +++ b/python/tests/test_restore.py @@ -1,15 +1,18 @@ import datetime import pathlib -import pyarrow as pa import pytest +from arro3.core import Table from deltalake import CommitProperties, DeltaTable, write_deltalake @pytest.mark.parametrize("use_relative", [True, False]) def test_restore_with_version( - tmp_path: pathlib.Path, sample_data: pa.Table, monkeypatch, use_relative: bool + tmp_path: pathlib.Path, + sample_table: Table, + monkeypatch, + use_relative: bool, ): if use_relative: monkeypatch.chdir(tmp_path) # Make tmp_path the working directory @@ -18,9 +21,9 @@ def test_restore_with_version( else: table_path = str(tmp_path) - write_deltalake(table_path, sample_data, mode="append") - write_deltalake(table_path, sample_data, mode="append") - write_deltalake(table_path, sample_data, mode="append") + write_deltalake(table_path, sample_table, mode="append") + write_deltalake(table_path, sample_table, mode="append") + write_deltalake(table_path, sample_table, mode="append") dt = DeltaTable(table_path) old_version = dt.version() @@ -34,7 +37,10 @@ def test_restore_with_version( @pytest.mark.parametrize("use_relative", [True, False]) def test_restore_with_datetime_str( - tmp_path: pathlib.Path, sample_data: pa.Table, monkeypatch, use_relative: bool + tmp_path: pathlib.Path, + sample_table: Table, + monkeypatch, + use_relative: bool, ): if use_relative: monkeypatch.chdir(tmp_path) # Make tmp_path the working directory @@ -43,9 +49,9 @@ def test_restore_with_datetime_str( else: table_path = str(tmp_path) - write_deltalake(table_path, sample_data, mode="append") - write_deltalake(table_path, sample_data, mode="append") - write_deltalake(table_path, sample_data, mode="append") + write_deltalake(table_path, sample_table, mode="append") + write_deltalake(table_path, sample_table, mode="append") + write_deltalake(table_path, sample_table, mode="append") dt = DeltaTable(table_path) old_version = dt.version() @@ -57,7 +63,10 @@ def test_restore_with_datetime_str( @pytest.mark.parametrize("use_relative", [True, False]) def test_restore_with_datetime( - tmp_path: pathlib.Path, sample_data: pa.Table, monkeypatch, use_relative: bool + tmp_path: pathlib.Path, + sample_table: Table, + monkeypatch, + use_relative: bool, ): if use_relative: monkeypatch.chdir(tmp_path) # Make tmp_path the working directory @@ -66,9 +75,9 @@ def test_restore_with_datetime( else: table_path = str(tmp_path) - write_deltalake(table_path, sample_data, mode="append") - write_deltalake(table_path, sample_data, mode="append") - write_deltalake(table_path, sample_data, mode="append") + write_deltalake(table_path, sample_table, mode="append") + write_deltalake(table_path, sample_table, mode="append") + write_deltalake(table_path, sample_table, mode="append") dt = DeltaTable(table_path) old_version = dt.version() diff --git a/python/tests/test_schema.py b/python/tests/test_schema.py index 56c5a75b00..eb9a49ed40 100644 --- a/python/tests/test_schema.py +++ b/python/tests/test_schema.py @@ -1,6 +1,5 @@ import json -import pyarrow as pa import pytest from deltalake import DeltaTable, Field @@ -11,10 +10,6 @@ Schema, StructType, ) -from deltalake.writer._conversion import ( - ArrowSchemaConversionMode, - _convert_pa_schema_to_delta, -) def test_table_schema(): @@ -39,30 +34,37 @@ def test_table_schema(): ) +@pytest.mark.pyarrow def test_table_schema_pyarrow_simple(): + import pyarrow as pa + table_path = "../crates/test/tests/data/simple_table" dt = DeltaTable(table_path) - schema = dt.schema().to_pyarrow() + schema = dt.schema().to_arrow() field = schema.field(0) assert len(schema.types) == 1 assert field.name == "id" assert field.type == pa.int64() assert field.nullable is True - assert field.metadata is None + assert field.metadata == {} +@pytest.mark.pyarrow def test_table_schema_pyarrow_020(): + import pyarrow as pa + table_path = "../crates/test/tests/data/delta-0.2.0" dt = DeltaTable(table_path) - schema = dt.schema().to_pyarrow() + schema = dt.schema().to_arrow() field = schema.field(0) assert len(schema.types) == 1 assert field.name == "value" assert field.type == pa.int32() assert field.nullable is True - assert field.metadata is None + assert field.metadata == {} +@pytest.mark.pyarrow def test_primitive_delta_types(): valid_types = [ "string", @@ -87,8 +89,8 @@ def test_primitive_delta_types(): assert data_type in str(delta_type) assert data_type in repr(delta_type) - pa_type = delta_type.to_pyarrow() - assert delta_type == PrimitiveType.from_pyarrow(pa_type) + pa_type = delta_type.to_arrow() + assert delta_type == PrimitiveType.from_arrow(pa_type) json_type = delta_type.to_json() assert delta_type == PrimitiveType.from_json(json_type) @@ -98,6 +100,7 @@ def test_primitive_delta_types(): PrimitiveType(data_type) +@pytest.mark.pyarrow def test_array_delta_types(): init_values = [ (PrimitiveType("string"), False), @@ -111,8 +114,8 @@ def test_array_delta_types(): assert array_type.element_type == element_type assert array_type.contains_null == contains_null - pa_type = array_type.to_pyarrow() - assert array_type == ArrayType.from_pyarrow(pa_type) + pa_type = array_type.to_arrow() + assert array_type == ArrayType.from_arrow(pa_type) json_type = array_type.to_json() assert array_type == ArrayType.from_json(json_type) @@ -138,13 +141,14 @@ def test_map_delta_types(): # Map type is not yet supported in C Data Interface # https://github.com/apache/arrow-rs/issues/2037 - # pa_type = map_type.to_pyarrow() - # assert map_type == PrimitiveType.from_pyarrow(pa_type) + # pa_type = map_type.to_arrow() + # assert map_type == PrimitiveType.from_arrow(pa_type) json_type = map_type.to_json() assert map_type == MapType.from_json(json_type) +@pytest.mark.pyarrow def test_struct_delta_types(): fields = [ Field("x", "integer", nullable=True, metadata={"x": {"y": 3}}), @@ -166,10 +170,11 @@ def test_struct_delta_types(): Field("y", PrimitiveType("string"), nullable=False), ] struct_type = StructType(fields) - pa_type = struct_type.to_pyarrow() - assert struct_type == StructType.from_pyarrow(pa_type) + pa_type = struct_type.to_arrow() + assert struct_type == StructType.from_arrow(pa_type) +@pytest.mark.pyarrow def test_delta_field(): args = [ ("x", PrimitiveType("string"), True, {}), @@ -193,13 +198,14 @@ def test_delta_field(): # Field metadata doesn't roundtrip currently # See: https://github.com/apache/arrow-rs/issues/478 if len(field.metadata) == 0: - pa_field = field.to_pyarrow() - assert field == Field.from_pyarrow(pa_field) + pa_field = field.to_arrow() + assert field == Field.from_arrow(pa_field) json_field = field.to_json() assert field == Field.from_json(json_field) +@pytest.mark.pyarrow def test_delta_schema(): fields = [ Field("x", "integer", nullable=True, metadata={"x": {"y": 3}}), @@ -211,8 +217,8 @@ def test_delta_schema(): assert schema.fields == fields empty_schema = Schema([]) - pa_schema = empty_schema.to_pyarrow() - assert empty_schema == Schema.from_pyarrow(pa_schema) + pa_schema = empty_schema.to_arrow() + assert empty_schema == Schema.from_arrow(pa_schema) # Field metadata doesn't roundtrip currently # See: https://github.com/apache/arrow-rs/issues/478 @@ -221,340 +227,8 @@ def test_delta_schema(): Field("y", ArrayType("string", contains_null=True), nullable=False), ] schema_without_metadata = schema = Schema(fields) - pa_schema = schema_without_metadata.to_pyarrow() - assert schema_without_metadata == Schema.from_pyarrow(pa_schema) - - -def _generate_test_type(): - class UuidType(pa.ExtensionType): - def __init__(self): - pa.ExtensionType.__init__(self, pa.binary(16), "my_package.uuid") - - def __arrow_ext_serialize__(self): - # since we don't have a parameterized type, we don't need extra - # metadata to be deserialized - return b"" - - @classmethod - def __arrow_ext_deserialize__(self, storage_type, serialized): - # return an instance of this subclass given the serialized - # metadata. - return UuidType() - - pa.register_extension_type(UuidType()) - return UuidType() - - -def _generate_test_tuples(): - test_tuples = [ - ( - pa.schema([("some_int", pa.uint32()), ("some_string", pa.string_view())]), - pa.schema([("some_int", pa.int32()), ("some_string", pa.string_view())]), - ArrowSchemaConversionMode.PASSTHROUGH, - ), - ( - pa.schema( - [ - ("some_int", pa.uint32()), - ("some_string", pa.list_view(pa.large_string())), - ] - ), - pa.schema( - [ - ("some_int", pa.int32()), - ("some_string", pa.list_view(pa.large_string())), - ] - ), - ArrowSchemaConversionMode.PASSTHROUGH, - ), - ( - pa.schema([("some_int", pa.uint32()), ("some_string", pa.string())]), - pa.schema([("some_int", pa.int32()), ("some_string", pa.string())]), - ArrowSchemaConversionMode.NORMAL, - ), - ( - pa.schema([("some_int", pa.uint32()), ("some_string", pa.string())]), - pa.schema([("some_int", pa.int32()), ("some_string", pa.string())]), - ArrowSchemaConversionMode.PASSTHROUGH, - ), - ( - pa.schema([("some_int", pa.uint32()), ("some_string", pa.large_string())]), - pa.schema([("some_int", pa.int32()), ("some_string", pa.large_string())]), - ArrowSchemaConversionMode.PASSTHROUGH, - ), - ( - pa.schema([("some_int", pa.uint32()), ("some_binary", pa.large_binary())]), - pa.schema([("some_int", pa.int32()), ("some_binary", pa.large_binary())]), - ArrowSchemaConversionMode.PASSTHROUGH, - ), - ( - pa.schema( - [ - ("some_int", pa.uint32()), - ("some_string", pa.large_list(pa.large_string())), - ] - ), - pa.schema( - [ - ("some_int", pa.int32()), - ("some_string", pa.large_list(pa.large_string())), - ] - ), - ArrowSchemaConversionMode.PASSTHROUGH, - ), - ( - pa.schema( - [ - ("some_int", pa.uint32()), - ("some_string", pa.list_(pa.large_string())), - ] - ), - pa.schema( - [("some_int", pa.int32()), ("some_string", pa.list_(pa.large_string()))] - ), - ArrowSchemaConversionMode.PASSTHROUGH, - ), - ( - pa.schema( - [ - pa.field("some_int", pa.uint32(), nullable=True), - pa.field("some_string", pa.string(), nullable=False), - pa.field("some_fixed_binary", pa.binary(5), nullable=False), - pa.field("some_decimal", pa.decimal128(10, 2), nullable=False), - ] - ), - pa.schema( - [ - pa.field("some_int", pa.int32(), nullable=True), - pa.field("some_string", pa.string(), nullable=False), - pa.field("some_fixed_binary", pa.binary(), nullable=False), - pa.field("some_decimal", pa.decimal128(10, 2), nullable=False), - ] - ), - ArrowSchemaConversionMode.NORMAL, - ), - ( - pa.schema( - [ - pa.field("some_int", pa.uint32(), nullable=True), - pa.field("some_string", pa.string(), nullable=False), - ] - ), - pa.schema( - [ - pa.field("some_int", pa.int32(), nullable=True), - pa.field("some_string", pa.large_string(), nullable=False), - ] - ), - ArrowSchemaConversionMode.LARGE, - ), - ( - pa.schema([("some_int", pa.uint32()), ("some_string", pa.string())]), - pa.schema([("some_int", pa.int32()), ("some_string", pa.large_string())]), - ArrowSchemaConversionMode.LARGE, - ), - ( - pa.schema([("some_int", pa.uint32()), ("some_string", pa.large_string())]), - pa.schema([("some_int", pa.int32()), ("some_string", pa.string())]), - ArrowSchemaConversionMode.NORMAL, - ), - ( - pa.schema( - [ - ("some_int", pa.uint8()), - ("some_int1", pa.uint16()), - ("some_int2", pa.uint32()), - ("some_int3", pa.uint64()), - ] - ), - pa.schema( - [ - ("some_int", pa.int8()), - ("some_int1", pa.int16()), - ("some_int2", pa.int32()), - ("some_int3", pa.int64()), - ] - ), - ArrowSchemaConversionMode.LARGE, - ), - ( - pa.schema( - [ - ("some_list", pa.list_(pa.string())), - ("some_fixed_list_int", pa.list_(pa.uint32(), 5)), - ("some_list_binary", pa.list_(pa.binary())), - ("some_string", pa.large_string()), - ] - ), - pa.schema( - [ - ("some_list", pa.large_list(pa.large_string())), - ("some_fixed_list_int", pa.large_list(pa.int32())), - ("some_list_binary", pa.large_list(pa.large_binary())), - ("some_string", pa.large_string()), - ] - ), - ArrowSchemaConversionMode.LARGE, - ), - ( - pa.schema( - [ - ("some_list", pa.large_list(pa.string())), - ("some_string", pa.large_string()), - ("some_binary", pa.large_binary()), - ] - ), - pa.schema( - [ - ("some_list", pa.list_(pa.string())), - ("some_string", pa.string()), - ("some_binary", pa.binary()), - ] - ), - ArrowSchemaConversionMode.NORMAL, - ), - ( - pa.schema( - [ - ("highly_nested_list", pa.list_(pa.list_(pa.list_(pa.string())))), - ( - "highly_nested_list_binary", - pa.list_(pa.list_(pa.list_(pa.binary()))), - ), - ("some_string", pa.large_string()), - ("some_binary", pa.large_binary()), - ] - ), - pa.schema( - [ - ( - "highly_nested_list", - pa.large_list(pa.large_list(pa.large_list(pa.large_string()))), - ), - ( - "highly_nested_list_binary", - pa.large_list(pa.large_list(pa.large_list(pa.large_binary()))), - ), - ("some_string", pa.large_string()), - ("some_binary", pa.large_binary()), - ] - ), - ArrowSchemaConversionMode.LARGE, - ), - ( - pa.schema( - [ - ( - "highly_nested_list", - pa.large_list(pa.list_(pa.large_list(pa.string()))), - ), - ( - "highly_nested_list_int", - pa.large_list(pa.list_(pa.large_list(pa.uint64()))), - ), - ("some_string", pa.large_string()), - ("some_binary", pa.large_binary()), - ] - ), - pa.schema( - [ - ("highly_nested_list", pa.list_(pa.list_(pa.list_(pa.string())))), - ( - "highly_nested_list_int", - pa.list_(pa.list_(pa.list_(pa.int64()))), - ), - ("some_string", pa.string()), - ("some_binary", pa.binary()), - ] - ), - ArrowSchemaConversionMode.NORMAL, - ), - ( - pa.schema( - [ - ("timestamp", pa.timestamp("s")), - ("timestamp1", pa.timestamp("ms")), - ("timestamp2", pa.timestamp("us")), - ("timestamp3", pa.timestamp("ns")), - ("timestamp4", pa.timestamp("s", tz="UTC")), - ("timestamp5", pa.timestamp("ms", tz="UTC")), - ("timestamp6", pa.timestamp("ns", tz="UTC")), - ("timestamp7", pa.timestamp("ns", tz="UTC")), - ] - ), - pa.schema( - [ - ("timestamp", pa.timestamp("us")), - ("timestamp1", pa.timestamp("us")), - ("timestamp2", pa.timestamp("us")), - ("timestamp3", pa.timestamp("us")), - ("timestamp4", pa.timestamp("us", tz="UTC")), - ("timestamp5", pa.timestamp("us", tz="UTC")), - ("timestamp6", pa.timestamp("us", tz="UTC")), - ("timestamp7", pa.timestamp("us", tz="UTC")), - ] - ), - ArrowSchemaConversionMode.NORMAL, - ), - ( - pa.schema( - [ - ( - "struct", - pa.struct( - { - "highly_nested_list": pa.large_list( - pa.list_(pa.large_list(pa.string())) - ), - "highly_nested_list_int": pa.large_list( - pa.list_(pa.large_list(pa.uint64())) - ), - "some_string": pa.large_string(), - "some_binary": pa.large_binary(), - } - ), - ) - ] - ), - pa.schema( - [ - ( - "struct", - pa.struct( - { - "highly_nested_list": pa.list_( - pa.list_(pa.list_(pa.string())) - ), - "highly_nested_list_int": pa.list_( - pa.list_(pa.list_(pa.int64())) - ), - "some_string": pa.string(), - "some_binary": pa.binary(), - } - ), - ) - ] - ), - ArrowSchemaConversionMode.NORMAL, - ), - ( - pa.schema([("uuid", _generate_test_type())]), - pa.schema([("uuid", pa.binary(16))]), - ArrowSchemaConversionMode.NORMAL, - ), - ] - - return test_tuples - - -@pytest.mark.parametrize( - "schema,expected_schema,conversion_mode", - _generate_test_tuples(), -) -def test_schema_conversions(schema, expected_schema, conversion_mode): - result_schema = _convert_pa_schema_to_delta(schema, conversion_mode) - - assert result_schema == expected_schema + pa_schema = schema_without_metadata.to_arrow() + assert schema_without_metadata == Schema.from_arrow(pa_schema) # diff --git a/python/tests/test_stats.py b/python/tests/test_stats.py index d4bc5e4978..8944077c14 100644 --- a/python/tests/test_stats.py +++ b/python/tests/test_stats.py @@ -1,6 +1,5 @@ from datetime import datetime, timezone -import pyarrow as pa import pytest from deltalake import DeltaTable, write_deltalake @@ -8,6 +7,7 @@ @pytest.mark.pandas @pytest.mark.polars +@pytest.mark.pyarrow def test_stats_usage_3201(tmp_path): # https://github.com/delta-io/delta-rs/issues/3201 # https://github.com/delta-io/delta-rs/issues/3173 @@ -71,8 +71,11 @@ def test_stats_usage_3201(tmp_path): assert_frame_equal(excepted, pl.from_arrow(data), check_row_order=False) +@pytest.mark.pyarrow @pytest.mark.parametrize("use_stats_struct", (True, False)) def test_microsecond_truncation_parquet_stats(tmp_path, use_stats_struct): + import pyarrow as pa + """In checkpoints the min,max value gets truncated to milliseconds precision. For min values this is not an issue, but for max values we need to round upwards. diff --git a/python/tests/test_table_read.py b/python/tests/test_table_read.py index 39cf10e7a8..a273b6e920 100644 --- a/python/tests/test_table_read.py +++ b/python/tests/test_table_read.py @@ -1,6 +1,7 @@ -import logging +import multiprocessing import os import tempfile +from concurrent.futures import Executor, ProcessPoolExecutor, ThreadPoolExecutor from datetime import date, datetime, timezone from pathlib import Path from random import random @@ -8,32 +9,23 @@ from typing import Any from unittest.mock import Mock +import pytest +from arro3.core import Array, DataType, Table +from arro3.core import Field as ArrowField + +from deltalake import DeltaTable from deltalake._util import encode_partition_value from deltalake.exceptions import DeltaProtocolError from deltalake.query import QueryBuilder from deltalake.table import ProtocolVersions from deltalake.writer import write_deltalake -try: - import pandas as pd -except ModuleNotFoundError: - _has_pandas = False -else: - _has_pandas = True - -import multiprocessing -from concurrent.futures import Executor, ProcessPoolExecutor, ThreadPoolExecutor - -import pyarrow as pa -import pyarrow.dataset as ds -import pytest -from pyarrow.dataset import ParquetReadOptions -from pyarrow.fs import LocalFileSystem, SubTreeFileSystem - -from deltalake import DeltaTable - +@pytest.mark.pyarrow def test_read_table_with_edge_timestamps(): + import pyarrow.dataset as ds + from pyarrow.dataset import ParquetReadOptions + table_path = "../crates/test/tests/data/table_with_edge_timestamps" dt = DeltaTable(table_path) dataset = dt.to_pyarrow_dataset( @@ -60,7 +52,9 @@ def test_read_table_with_edge_timestamps(): def test_read_simple_table_to_dict(): table_path = "../crates/test/tests/data/simple_table" dt = DeltaTable(table_path) - assert dt.to_pyarrow_dataset().to_table().to_pydict() == {"id": [5, 7, 9]} + assert QueryBuilder().register("tbl", dt).execute("select * from tbl").read_all()[ + "id" + ].to_pylist() == [5, 7, 9] class _SerializableException(BaseException): @@ -77,7 +71,7 @@ def _recursively_read_simple_table(executor_class: type[Executor], depth): if depth == 0: return # We use concurrent.futures.Executors instead of `threading.Thread` or `multiprocessing.Process` to that errors - # are re-rasied in the parent process/thread when we call `future.result()`. + # are re-raised in the parent process/thread when we call `future.result()`. with executor_class(max_workers=1) as executor: future = executor.submit( _recursively_read_simple_table, executor_class, depth - 1 @@ -109,12 +103,14 @@ def test_read_simple_in_threads_and_processes( _recursively_read_simple_table(executor_class=executor_class, depth=5) +@pytest.mark.pyarrow def test_read_simple_table_by_version_to_dict(): table_path = "../crates/test/tests/data/delta-0.2.0" dt = DeltaTable(table_path, version=2) assert dt.to_pyarrow_dataset().to_table().to_pydict() == {"value": [1, 2, 3]} +@pytest.mark.pyarrow def test_read_simple_table_using_options_to_dict(): table_path = "../crates/test/tests/data/delta-0.2.0" dt = DeltaTable(table_path, version=2, storage_options={}) @@ -210,6 +206,7 @@ def test_load_as_version_datetime_bad_format(): dt.load_as_version(bad_format) +@pytest.mark.pyarrow def test_read_simple_table_update_incremental(): table_path = "../crates/test/tests/data/simple_table" dt = DeltaTable(table_path, version=0) @@ -218,14 +215,15 @@ def test_read_simple_table_update_incremental(): assert dt.to_pyarrow_dataset().to_table().to_pydict() == {"id": [5, 7, 9]} +@pytest.mark.pyarrow def test_read_simple_table_file_sizes_failure(mocker): table_path = "../crates/test/tests/data/simple_table" dt = DeltaTable(table_path) - add_actions = dt.get_add_actions().to_pydict() + add_actions = dt.get_add_actions() # set all sizes to -1, the idea is to break the reading, to check # that input file sizes are actually used - add_actions_modified = {x: -1 for x in add_actions["path"]} + add_actions_modified = {x.as_py(): -1 for x in add_actions["path"]} mocker.patch( "deltalake._internal.RawDeltaTable.get_add_file_sizes", return_value=add_actions_modified, @@ -235,6 +233,7 @@ def test_read_simple_table_file_sizes_failure(mocker): dt.to_pyarrow_dataset().to_table().to_pydict() +@pytest.mark.pyarrow def test_read_partitioned_table_to_dict(): table_path = "../crates/test/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) @@ -247,6 +246,7 @@ def test_read_partitioned_table_to_dict(): assert dt.to_pyarrow_dataset().to_table().to_pydict() == expected +@pytest.mark.pyarrow def test_read_partitioned_table_with_partitions_filters_to_dict(): table_path = "../crates/test/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) @@ -261,6 +261,7 @@ def test_read_partitioned_table_with_partitions_filters_to_dict(): assert dt.to_pyarrow_dataset(partitions).to_table().to_pydict() == expected +@pytest.mark.pyarrow def test_read_empty_delta_table_after_delete(): table_path = "../crates/test/tests/data/delta-0.8-empty" dt = DeltaTable(table_path) @@ -269,6 +270,7 @@ def test_read_empty_delta_table_after_delete(): assert dt.to_pyarrow_dataset().to_table().to_pydict() == expected +@pytest.mark.pyarrow def test_read_table_with_column_subset(): table_path = "../crates/test/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) @@ -282,11 +284,15 @@ def test_read_table_with_column_subset(): ) +@pytest.mark.pyarrow def test_read_table_as_category(): + import pyarrow as pa + import pyarrow.dataset as ds + table_path = "../crates/test/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) - assert dt.schema().to_pyarrow().field("value").type == pa.string() + assert dt.schema().to_arrow().field("value").type == pa.string() read_options = ds.ParquetReadOptions(dictionary_columns={"value"}) @@ -296,7 +302,10 @@ def test_read_table_as_category(): assert data.schema.field("day").type == pa.string() +@pytest.mark.pyarrow def test_read_table_with_filter(): + import pyarrow.dataset as ds + table_path = "../crates/test/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) expected = { @@ -313,7 +322,10 @@ def test_read_table_with_filter(): assert dataset.to_table(filter=filter_expr).to_pydict() == expected +@pytest.mark.pyarrow def test_read_table_with_stats(): + import pyarrow.dataset as ds + table_path = "../crates/test/tests/data/COVID-19_NYT" dt = DeltaTable(table_path) dataset = dt.to_pyarrow_dataset() @@ -337,6 +349,7 @@ def test_read_table_with_stats(): assert data.num_rows == 0 +@pytest.mark.pyarrow def test_read_special_partition(): table_path = "../crates/test/tests/data/delta-0.8.0-special-partition" dt = DeltaTable(table_path) @@ -379,13 +392,15 @@ def test_read_partitioned_table_protocol(): assert protocol.min_writer_version == 2 +@pytest.mark.pyarrow def test_read_table_with_cdc(): table_path = "../crates/test/tests/data/simple_table_with_cdc" dt = DeltaTable(table_path) - assert dt.to_pyarrow_table().to_pydict() == { - "id": [0], - "name": ["Mino"], - } + + result = QueryBuilder().register("tbl", dt).execute("select * from tbl").read_all() + + assert result["id"].to_pylist() == [0] + assert result["name"].to_pylist() == ["Mino"] def test_history_partitioned_table_metadata(): @@ -413,12 +428,15 @@ def test_history_partitioned_table_metadata(): @pytest.mark.parametrize("flatten", [True, False]) +@pytest.mark.pyarrow def test_add_actions_table(flatten: bool): + import pyarrow as pa + table_path = "../crates/test/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) actions_df = dt.get_add_actions(flatten) # RecordBatch doesn't have a sort_by method yet - actions_df = pa.Table.from_batches([actions_df]).sort_by("path").to_batches()[0] + actions_df = pa.table(actions_df).sort_by("path").to_batches()[0] assert actions_df.num_rows == 6 assert actions_df["path"] == pa.array( @@ -533,22 +551,32 @@ def test_get_files_partitioned_table(): @pytest.mark.pandas +@pytest.mark.pyarrow def test_delta_table_to_pandas(): + import pandas as pd + table_path = "../crates/test/tests/data/simple_table" dt = DeltaTable(table_path) assert dt.to_pandas().equals(pd.DataFrame({"id": [5, 7, 9]})) @pytest.mark.pandas +@pytest.mark.pyarrow def test_delta_table_with_filesystem(): + import pandas as pd + from pyarrow.fs import LocalFileSystem, SubTreeFileSystem + table_path = "../crates/test/tests/data/simple_table" dt = DeltaTable(table_path) filesystem = SubTreeFileSystem(table_path, LocalFileSystem()) assert dt.to_pandas(filesystem=filesystem).equals(pd.DataFrame({"id": [5, 7, 9]})) +@pytest.mark.pyarrow @pytest.mark.pandas def test_delta_table_with_filters(): + import pyarrow.dataset as ds + table_path = "../crates/test/tests/data/COVID-19_NYT" dt = DeltaTable(table_path) dataset = dt.to_pyarrow_dataset() @@ -596,7 +624,10 @@ def test_delta_table_with_filters(): ) +@pytest.mark.pyarrow def test_writer_fails_on_protocol(): + import pytest + table_path = "../crates/test/tests/data/simple_table" dt = DeltaTable(table_path) dt.protocol = Mock(return_value=ProtocolVersions(2, 1, None, None)) @@ -699,7 +730,11 @@ def assert_num_fragments(table, predicate, count): assert len(list(frags)) == count +@pytest.mark.pyarrow def test_filter_nulls(tmp_path: Path): + import pyarrow as pa + import pyarrow.dataset as ds + def assert_scan_equals(table, predicate, expected): data = table.to_pyarrow_dataset().to_table(filter=predicate).sort_by("part") assert data == expected @@ -766,7 +801,10 @@ def assert_scan_equals(table, predicate, expected): assert_scan_equals(table, predicate, expected) +@pytest.mark.pyarrow def test_issue_1653_filter_bool_partition(tmp_path: Path): + import pyarrow as pa + ta = pa.Table.from_pydict( { "bool_col": [True, False, True, False], @@ -874,7 +912,10 @@ def test_partitions_filtering_partitioned_table(): partition in actual +@pytest.mark.pyarrow def test_partitions_date_partitioned_table(): + import pyarrow as pa + table_path = tempfile.gettempdir() + "/date_partition_table" date_partitions = [ date(2024, 8, 1), @@ -882,14 +923,14 @@ def test_partitions_date_partitioned_table(): date(2024, 8, 3), date(2024, 8, 4), ] - sample_data = pa.table( + sample_data_pyarrow = pa.table( { "date_field": pa.array(date_partitions, pa.date32()), "numeric_data": pa.array([1, 2, 3, 4], pa.int64()), } ) write_deltalake( - table_path, sample_data, mode="overwrite", partition_by=["date_field"] + table_path, sample_data_pyarrow, mode="overwrite", partition_by=["date_field"] ) delta_table = DeltaTable(table_path) @@ -950,47 +991,81 @@ def test_is_deltatable_with_storage_opts(): assert DeltaTable.is_deltatable(table_path, storage_options=storage_options) +@pytest.mark.pyarrow +@pytest.mark.xfail( + reason="Issue: columns are encoded as dictionary due to TableProvider" +) def test_read_query_builder(): table_path = "../crates/test/tests/data/delta-0.8.0-partitioned" dt = DeltaTable(table_path) - expected = { - "value": ["4", "5", "6", "7"], - "year": ["2021", "2021", "2021", "2021"], - "month": ["4", "12", "12", "12"], - "day": ["5", "4", "20", "20"], - } + + expected = Table( + { + "value": Array( + ["4", "5", "6", "7"], + ArrowField("value", type=DataType.string(), nullable=True), + ), + "year": Array( + ["2021", "2021", "2021", "2021"], + ArrowField("year", type=DataType.string(), nullable=True), + ), + "month": Array( + ["4", "12", "12", "12"], + ArrowField("month", type=DataType.string(), nullable=True), + ), + "day": Array( + ["5", "4", "20", "20"], + ArrowField("day", type=DataType.string(), nullable=True), + ), + } + ) qb = QueryBuilder().register("tbl", dt) query = "SELECT * FROM tbl WHERE year >= 2021 ORDER BY value" - actual = pa.Table.from_batches(qb.execute(query).fetchall()).to_pydict() - assert expected == actual - - actual = pa.Table.from_batches(qb.sql(query).fetchall()).to_pydict() + actual = qb.execute(query).read_all() assert expected == actual +@pytest.mark.pyarrow def test_read_query_builder_join_multiple_tables(tmp_path): table_path = "../crates/test/tests/data/delta-0.8.0-date" dt1 = DeltaTable(table_path) write_deltalake( tmp_path, - pa.table( + data=Table( { - "date": ["2021-01-01", "2021-01-02", "2021-01-03", "2021-12-31"], - "value": ["a", "b", "c", "d"], + "date": Array( + ["2021-01-01", "2021-01-02", "2021-01-03", "2021-12-31"], + ArrowField("date", type=DataType.string(), nullable=True), + ), + "value": Array( + ["a", "b", "c", "d"], + ArrowField("value", type=DataType.string(), nullable=True), + ), } ), ) dt2 = DeltaTable(tmp_path) - expected = { - "date": ["2021-01-01", "2021-01-02", "2021-01-03"], - "dayOfYear": [1, 2, 3], - "value": ["a", "b", "c"], - } - actual = pa.Table.from_batches( + expected = Table( + { + "date": Array( + ["2021-01-01", "2021-01-02", "2021-01-03"], + ArrowField("date", type=DataType.string(), nullable=True), + ), + "dayOfYear": Array( + [1, 2, 3], + ArrowField("dayOfYear", type=DataType.int32(), nullable=True), + ), + "value": Array( + ["a", "b", "c"], + ArrowField("value", type=DataType.string(), nullable=True), + ), + } + ) + actual = ( QueryBuilder() .register("tbl1", dt1) .register("tbl2", dt2) @@ -1002,22 +1077,6 @@ def test_read_query_builder_join_multiple_tables(tmp_path): ORDER BY tbl1.date """ ) - .fetchall() - ).to_pydict() + .read_all() + ) assert expected == actual - - -def test_read_query_builder_show_output(capsys, caplog): - table_path = "../crates/test/tests/data/delta-0.8.0-partitioned" - dt = DeltaTable(table_path) - logging.getLogger("deltalake").setLevel(logging.INFO) - - qb = QueryBuilder().register("tbl", dt) - query = "SELECT * FROM tbl WHERE year >= 2021 ORDER BY value" - qb.execute(query).show() - assert capsys.readouterr().out.strip() != "" - - query = "SELECT * FROM tbl WHERE year >= 9999" - qb.execute(query).show() - assert "query contains no records" in caplog.text - assert capsys.readouterr().out.strip() == "" diff --git a/python/tests/test_threaded.py b/python/tests/test_threaded.py index 9839e79713..229179fb2e 100644 --- a/python/tests/test_threaded.py +++ b/python/tests/test_threaded.py @@ -6,15 +6,19 @@ import pathlib import threading from concurrent.futures import ThreadPoolExecutor +from typing import TYPE_CHECKING -import pyarrow as pa import pytest from deltalake import DeltaTable, write_deltalake from deltalake.exceptions import CommitFailedError +if TYPE_CHECKING: + import pyarrow as pa -def test_concurrency(existing_table: DeltaTable, sample_data: pa.Table): + +@pytest.mark.pyarrow +def test_concurrency(existing_table: DeltaTable, sample_data_pyarrow: "pa.Table"): exception = None def comp(): @@ -25,9 +29,9 @@ def comp(): data = DeltaTable(dt.table_uri).to_pyarrow_table() # If two overwrites delete the same file and then add their own # concurrently, then this will fail. - assert data.num_rows == sample_data.num_rows + assert data.num_rows == sample_data_pyarrow.num_rows try: - write_deltalake(dt.table_uri, sample_data, mode="overwrite") + write_deltalake(dt.table_uri, sample_data_pyarrow, mode="overwrite") except Exception as e: exception = e @@ -47,10 +51,10 @@ def comp(): @pytest.mark.polars -def test_multithreaded_write_using_table(sample_data: pa.Table, tmp_path: pathlib.Path): +def test_multithreaded_write_using_table(tmp_path: pathlib.Path): import polars as pl - table = pl.DataFrame({"a": [1, 2, 3]}).to_arrow() + table = pl.DataFrame({"a": [1, 2, 3]}) write_deltalake(tmp_path, table, mode="overwrite") dt = DeltaTable(tmp_path) @@ -61,10 +65,11 @@ def test_multithreaded_write_using_table(sample_data: pa.Table, tmp_path: pathli @pytest.mark.polars -def test_multithreaded_write_using_path(sample_data: pa.Table, tmp_path: pathlib.Path): +@pytest.mark.xfail(reason="Can fail because of already borrowed") +def test_multithreaded_write_using_path(tmp_path: pathlib.Path): import polars as pl - table = pl.DataFrame({"a": [1, 2, 3]}).to_arrow() + table = pl.DataFrame({"a": [1, 2, 3]}) write_deltalake(tmp_path, table, mode="overwrite") with ThreadPoolExecutor() as exe: diff --git a/python/tests/test_unity_catalog.py b/python/tests/test_unity_catalog.py index a1c5c45495..d5458ab317 100644 --- a/python/tests/test_unity_catalog.py +++ b/python/tests/test_unity_catalog.py @@ -7,6 +7,7 @@ @pytest.mark.unitycatalog_databricks +@pytest.mark.pyarrow @pytest.mark.integration @pytest.mark.timeout(timeout=10, method="thread") def test_uc_read_deltatable(): @@ -30,6 +31,7 @@ def test_uc_read_deltatable(): @pytest.mark.unitycatalog_oss @pytest.mark.integration +@pytest.mark.pyarrow @pytest.mark.timeout(timeout=10, method="thread") def test_uc_read_deltatable_failing(): """Test delta table reads using Unity Catalog URL (uc://)""" diff --git a/python/tests/test_update.py b/python/tests/test_update.py index 199248ea22..bc709ec954 100644 --- a/python/tests/test_update.py +++ b/python/tests/test_update.py @@ -1,44 +1,69 @@ import pathlib -import pyarrow as pa import pytest +from arro3.core import Array, DataType, Table +from arro3.core import Field as ArrowField from deltalake import CommitProperties, DeltaTable, write_deltalake -from deltalake.writer._conversion import ( - ArrowSchemaConversionMode, - convert_pyarrow_table, -) +from deltalake.query import QueryBuilder @pytest.fixture() def sample_table(): nrows = 5 - return pa.table( + return Table( { - "id": pa.array(["1", "2", "3", "4", "5"]), - "price": pa.array(list(range(nrows)), pa.int64()), - "sold": pa.array(list(range(nrows)), pa.int64()), - "price_float": pa.array(list(range(nrows)), pa.float64()), - "items_in_bucket": pa.array([["item1", "item2", "item3"]] * nrows), - "deleted": pa.array([False] * nrows), + "id": Array( + ["1", "2", "3", "4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + list(range(nrows)), + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + list(range(nrows)), + ArrowField("sold", type=DataType.int64(), nullable=True), + ), + "price_float": Array( + list(range(nrows)), + ArrowField("price_float", type=DataType.float64(), nullable=True), + ), + "deleted": Array( + [False] * nrows, + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) -def test_update_with_predicate(tmp_path: pathlib.Path, sample_table: pa.Table): +def test_update_with_predicate(tmp_path: pathlib.Path, sample_table: Table): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - nrows = 5 - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5"]), - "price": pa.array(list(range(nrows)), pa.int64()), - "sold": pa.array(list(range(nrows)), pa.int64()), - "price_float": pa.array(list(range(nrows)), pa.float64()), - "items_in_bucket": pa.array([["item1", "item2", "item3"]] * nrows), - "deleted": pa.array([False, False, False, False, True]), + "id": Array( + ["1", "2", "3", "4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 3, 4], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 3, 4], + ArrowField("sold", type=DataType.int64(), nullable=True), + ), + "price_float": Array( + [0.0, 1.0, 2.0, 3.0, 4.0], + ArrowField("price_float", type=DataType.float64(), nullable=True), + ), + "deleted": Array( + [False, False, False, False, True], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) @@ -49,7 +74,7 @@ def test_update_with_predicate(tmp_path: pathlib.Path, sample_table: pa.Table): commit_properties=commit_properties, ) - result = dt.to_pyarrow_table() + result = QueryBuilder().register("tbl", dt).execute("select * from tbl").read_all() last_action = dt.history(1)[0] assert last_action["operation"] == "UPDATE" @@ -57,67 +82,46 @@ def test_update_with_predicate(tmp_path: pathlib.Path, sample_table: pa.Table): assert result == expected -def test_update_with_predicate_large_dtypes( - tmp_path: pathlib.Path, sample_table: pa.Table -): - sample_table = convert_pyarrow_table( - sample_table, schema_conversion_mode=ArrowSchemaConversionMode.LARGE - ) +def test_update_wo_predicate(tmp_path: pathlib.Path, sample_table: Table): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - nrows = 5 - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "5"]), - "price": pa.array(list(range(nrows)), pa.int64()), - "sold": pa.array(list(range(nrows)), pa.int64()), - "price_float": pa.array(list(range(nrows)), pa.float64()), - "items_in_bucket": pa.array([["item1", "item2", "item3"]] * nrows), - "deleted": pa.array([True, False, False, False, False]), - } - ) - - dt.update( - updates={"deleted": "True"}, - predicate="id = '1'", - ) - - result = dt.to_pyarrow_table() - last_action = dt.history(1)[0] - - assert last_action["operation"] == "UPDATE" - assert result == expected - - -def test_update_wo_predicate(tmp_path: pathlib.Path, sample_table: pa.Table): - write_deltalake(tmp_path, sample_table, mode="append") - - dt = DeltaTable(tmp_path) - - nrows = 5 - expected = pa.table( - { - "id": pa.array(["1", "2", "3", "4", "5"]), - "price": pa.array(list(range(nrows)), pa.int64()), - "sold": pa.array(list(range(nrows)), pa.int64()), - "price_float": pa.array(list(range(nrows)), pa.float64()), - "items_in_bucket": pa.array([["item1", "item2", "item3"]] * nrows), - "deleted": pa.array([True] * 5), + "id": Array( + ["1", "2", "3", "4", "5"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 3, 4], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 3, 4], + ArrowField("sold", type=DataType.int64(), nullable=True), + ), + "price_float": Array( + [0.0, 1.0, 2.0, 3.0, 4.0], + ArrowField("price_float", type=DataType.float64(), nullable=True), + ), + "deleted": Array( + [True, True, True, True, True], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) dt.update(updates={"deleted": "True"}) - result = dt.to_pyarrow_table() + result = QueryBuilder().register("tbl", dt).execute("select * from tbl").read_all() last_action = dt.history(1)[0] assert last_action["operation"] == "UPDATE" assert result == expected -def test_update_wrong_types_cast(tmp_path: pathlib.Path, sample_table: pa.Table): +def test_update_wrong_types_cast(tmp_path: pathlib.Path, sample_table: Table): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) @@ -130,20 +134,34 @@ def test_update_wrong_types_cast(tmp_path: pathlib.Path, sample_table: pa.Table) def test_update_wo_predicate_multiple_updates( - tmp_path: pathlib.Path, sample_table: pa.Table + tmp_path: pathlib.Path, sample_table: Table ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - expected = pa.table( + expected = Table( { - "id": pa.array(["1_1", "2_1", "3_1", "4_1", "5_1"]), - "price": pa.array([0, 1, 2, 3, 4], pa.int64()), - "sold": pa.array([0, 1, 4, 9, 16], pa.int64()), - "price_float": pa.array(list(range(5)), pa.float64()), - "items_in_bucket": pa.array([["item1", "item2", "item3"]] * 5), - "deleted": pa.array([True] * 5), + "id": Array( + ["1_1", "2_1", "3_1", "4_1", "5_1"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 3, 4], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 4, 9, 16], + ArrowField("sold", type=DataType.int64(), nullable=True), + ), + "price_float": Array( + [0.0, 1.0, 2.0, 3.0, 4.0], + ArrowField("price_float", type=DataType.float64(), nullable=True), + ), + "deleted": Array( + [True, True, True, True, True], + ArrowField("deleted", type=DataType.bool(), nullable=True), + ), } ) @@ -152,7 +170,7 @@ def test_update_wo_predicate_multiple_updates( error_on_type_mismatch=False, ) - result = dt.to_pyarrow_table() + result = QueryBuilder().register("tbl", dt).execute("select * from tbl").read_all() last_action = dt.history(1)[0] assert last_action["operation"] == "UPDATE" @@ -160,23 +178,34 @@ def test_update_wo_predicate_multiple_updates( def test_update_with_predicate_and_new_values( - tmp_path: pathlib.Path, sample_table: pa.Table + tmp_path: pathlib.Path, sample_table: Table ): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) - nrows = 5 - expected = pa.table( + expected = Table( { - "id": pa.array(["1", "2", "3", "4", "new_id"]), - "price": pa.array(list(range(nrows)), pa.int64()), - "sold": pa.array([0, 1, 2, 3, 100], pa.int64()), - "price_float": pa.array([0, 1, 2, 3, 9999], pa.float64()), - "items_in_bucket": pa.array( - [["item1", "item2", "item3"]] * 4 + [["item4", "item5", "item6"]] + "id": Array( + ["1", "2", "3", "4", "new_id"], + ArrowField("id", type=DataType.string(), nullable=True), + ), + "price": Array( + [0, 1, 2, 3, 4], + ArrowField("price", type=DataType.int64(), nullable=True), + ), + "sold": Array( + [0, 1, 2, 3, 100], + ArrowField("sold", type=DataType.int64(), nullable=True), + ), + "price_float": Array( + [0.0, 1.0, 2.0, 3.0, 9999.0], + ArrowField("price_float", type=DataType.float64(), nullable=True), + ), + "deleted": Array( + [False, False, False, False, True], + ArrowField("deleted", type=DataType.bool(), nullable=True), ), - "deleted": pa.array([False, False, False, False, True]), } ) @@ -191,14 +220,14 @@ def test_update_with_predicate_and_new_values( predicate="price > 3", ) - result = dt.to_pyarrow_table() + result = QueryBuilder().register("tbl", dt).execute("select * from tbl").read_all() last_action = dt.history(1)[0] assert last_action["operation"] == "UPDATE" assert result == expected -def test_update_no_inputs(tmp_path: pathlib.Path, sample_table: pa.Table): +def test_update_no_inputs(tmp_path: pathlib.Path, sample_table: Table): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) @@ -212,7 +241,7 @@ def test_update_no_inputs(tmp_path: pathlib.Path, sample_table: pa.Table): ) -def test_update_to_many_inputs(tmp_path: pathlib.Path, sample_table: pa.Table): +def test_update_to_many_inputs(tmp_path: pathlib.Path, sample_table: Table): write_deltalake(tmp_path, sample_table, mode="append") dt = DeltaTable(tmp_path) @@ -227,7 +256,7 @@ def test_update_to_many_inputs(tmp_path: pathlib.Path, sample_table: pa.Table): def test_update_with_incorrect_updates_input( - tmp_path: pathlib.Path, sample_table: pa.Table + tmp_path: pathlib.Path, sample_table: Table ): write_deltalake(tmp_path, sample_table, mode="append") @@ -243,13 +272,23 @@ def test_update_with_incorrect_updates_input( def test_update_stats_columns_stats_provided(tmp_path: pathlib.Path): - data = pa.table( + data = Table( { - "foo": pa.array(["a", "b", None, None]), - "bar": pa.array([1, 2, 3, None]), - "baz": pa.array([1, 1, None, None]), + "foo": Array( + ["a", "b", None, None], + ArrowField("foo", type=DataType.string(), nullable=True), + ), + "bar": Array( + [1, 2, 3, None], + ArrowField("bar", type=DataType.int64(), nullable=True), + ), + "baz": Array( + [1, 1, None, None], + ArrowField("baz", type=DataType.int64(), nullable=True), + ), } ) + write_deltalake( tmp_path, data, @@ -258,31 +297,36 @@ def test_update_stats_columns_stats_provided(tmp_path: pathlib.Path): ) dt = DeltaTable(tmp_path) add_actions_table = dt.get_add_actions(flatten=True) - stats = add_actions_table.to_pylist()[0] - - assert stats["null_count.foo"] == 2 - assert stats["min.foo"] == "a" - assert stats["max.foo"] == "b" - assert stats["null_count.bar"] is None - assert stats["min.bar"] is None - assert stats["max.bar"] is None - assert stats["null_count.baz"] == 2 - assert stats["min.baz"] == 1 - assert stats["max.baz"] == 1 + + def get_value(name: str): + return add_actions_table.column(name)[0].as_py() + + # x1 has no max, since inf was the highest value + assert get_value("null_count.foo") == 2 + assert get_value("min.foo") == "a" + assert get_value("max.foo") == "b" + assert get_value("null_count.bar") is None + assert get_value("min.bar") is None + assert get_value("max.bar") is None + assert get_value("null_count.baz") == 2 + assert get_value("min.baz") == 1 + assert get_value("max.baz") == 1 dt.update({"foo": "'hello world'"}) dt = DeltaTable(tmp_path) add_actions_table = dt.get_add_actions(flatten=True) - stats = add_actions_table.to_pylist()[0] - - assert dt.version() == 1 - assert stats["null_count.foo"] == 0 - assert stats["min.foo"] == "hello world" - assert stats["max.foo"] == "hello world" - assert stats["null_count.bar"] is None - assert stats["min.bar"] is None - assert stats["max.bar"] is None - assert stats["null_count.baz"] == 2 - assert stats["min.baz"] == 1 - assert stats["max.baz"] == 1 + + def get_value(name: str): + return add_actions_table.column(name)[0].as_py() + + # x1 has no max, since inf was the highest value + assert get_value("null_count.foo") == 0 + assert get_value("min.foo") == "hello world" + assert get_value("max.foo") == "hello world" + assert get_value("null_count.bar") is None + assert get_value("min.bar") is None + assert get_value("max.bar") is None + assert get_value("null_count.baz") == 2 + assert get_value("min.baz") == 1 + assert get_value("max.baz") == 1 diff --git a/python/tests/test_vacuum.py b/python/tests/test_vacuum.py index 6150accf90..5d7512daff 100644 --- a/python/tests/test_vacuum.py +++ b/python/tests/test_vacuum.py @@ -1,8 +1,8 @@ import os import pathlib -import pyarrow as pa import pytest +from arro3.core import Table from deltalake import CommitProperties, DeltaTable, write_deltalake @@ -37,7 +37,10 @@ def test_vacuum_dry_run_simple_table(): @pytest.mark.parametrize("use_relative", [True, False]) def test_vacuum_zero_duration( - tmp_path: pathlib.Path, sample_data: pa.Table, monkeypatch, use_relative: bool + tmp_path: pathlib.Path, + sample_table: Table, + monkeypatch, + use_relative: bool, ): if use_relative: monkeypatch.chdir(tmp_path) # Make tmp_path the working directory @@ -46,10 +49,10 @@ def test_vacuum_zero_duration( else: table_path = str(tmp_path) - write_deltalake(table_path, sample_data, mode="overwrite") + write_deltalake(table_path, sample_table, mode="overwrite") dt = DeltaTable(table_path) original_files = set(dt.files()) - write_deltalake(table_path, sample_data, mode="overwrite") + write_deltalake(table_path, sample_table, mode="overwrite") dt.update_incremental() new_files = set(dt.files()) assert new_files.isdisjoint(original_files) @@ -66,9 +69,9 @@ def test_vacuum_zero_duration( assert parquet_files == new_files -def test_vacuum_transaction_log(tmp_path: pathlib.Path, sample_data: pa.Table): +def test_vacuum_transaction_log(tmp_path: pathlib.Path, sample_table: Table): for i in range(5): - write_deltalake(tmp_path, sample_data, mode="overwrite") + write_deltalake(tmp_path, sample_table, mode="overwrite") dt = DeltaTable(tmp_path) diff --git a/python/tests/test_writer.py b/python/tests/test_writer.py index 9196a425da..dd5d3e6f4b 100644 --- a/python/tests/test_writer.py +++ b/python/tests/test_writer.py @@ -3,40 +3,30 @@ import pathlib from datetime import date, datetime, timezone from decimal import Decimal -from math import inf -from typing import Any +from typing import TYPE_CHECKING -import pyarrow as pa -import pyarrow.compute as pc import pytest -from pyarrow.dataset import dataset -from pyarrow.lib import RecordBatchReader +from arro3.core import Array, ChunkedArray, DataType, RecordBatchReader, Table +from arro3.core import Field as ArrowField from deltalake import CommitProperties, DeltaTable, Transaction, write_deltalake +from deltalake._internal import Field, PrimitiveType, Schema, StructType from deltalake.exceptions import ( DeltaError, DeltaProtocolError, SchemaMismatchError, ) -from deltalake.writer._conversion import ( - ArrowSchemaConversionMode, - convert_pyarrow_recordbatchreader, - convert_pyarrow_table, -) +from deltalake.query import QueryBuilder from deltalake.writer._utils import try_get_table_and_table_uri -try: - from pandas.testing import assert_frame_equal -except ModuleNotFoundError: - _has_pandas = False -else: - _has_pandas = True +if TYPE_CHECKING: + import pyarrow as pa @pytest.mark.skip(reason="Waiting on #570") def test_handle_existing( tmp_path: pathlib.Path, - sample_data: pa.Table, + sample_data: Table, ): # if uri points to a non-empty directory that isn't a delta table, error tmp_path @@ -49,23 +39,26 @@ def test_handle_existing( assert "directory is not empty" in str(exception) +@pytest.mark.pyarrow def test_roundtrip_basic( tmp_path: pathlib.Path, - sample_data: pa.Table, + sample_data_pyarrow: "pa.Table", ): # Check we can create the subdirectory + import pyarrow as pa + tmp_path = tmp_path / "path" / "to" / "table" start_time = datetime.now().timestamp() - write_deltalake(tmp_path, sample_data) + write_deltalake(tmp_path, sample_data_pyarrow) end_time = datetime.now().timestamp() assert ("0" * 20 + ".json") in os.listdir(tmp_path / "_delta_log") delta_table = DeltaTable(tmp_path) - assert delta_table.schema().to_pyarrow() == sample_data.schema + assert pa.schema(delta_table.schema()) == sample_data_pyarrow.schema table = delta_table.to_pyarrow_table() - assert table == sample_data + assert table == sample_data_pyarrow for add_path in get_add_paths(delta_table): # Paths should be relative, and with no partitioning have no directories @@ -82,20 +75,27 @@ def test_roundtrip_basic( @pytest.mark.parametrize("mode", ["append", "overwrite"]) -def test_enforce_schema_rust_writer(existing_table: DeltaTable, mode: str): - bad_data = pa.table({"x": pa.array([1, 2, 3])}) +def test_enforce_schema_rust_writer(existing_sample_table: DeltaTable, mode: str): + bad_data = Table( + { + "x": Array( + [1, 2, 3], + ArrowField("x", type=DataType.int64(), nullable=False), + ), + } + ) with pytest.raises( SchemaMismatchError, match=".*Cannot cast schema, number of fields does not match.*", ): write_deltalake( - existing_table, - bad_data, + table_or_uri=existing_sample_table, + data=bad_data, mode=mode, ) - table_uri = existing_table._table.table_uri() + table_uri = existing_sample_table._table.table_uri() with pytest.raises( SchemaMismatchError, match=".*Cannot cast schema, number of fields does not match.*", @@ -107,23 +107,38 @@ def test_enforce_schema_rust_writer(existing_table: DeltaTable, mode: str): ) -def test_update_schema(existing_table: DeltaTable): - new_data = pa.table({"x": pa.array([1, 2, 3])}) +def test_update_schema(existing_sample_table: DeltaTable): + new_data = Table( + { + "x": Array( + [1, 2, 3], + ArrowField("x", type=DataType.int64(), nullable=False), + ), + } + ) with pytest.raises(DeltaError): write_deltalake( - existing_table, new_data, mode="append", schema_mode="overwrite" + existing_sample_table, new_data, mode="append", schema_mode="overwrite" ) - write_deltalake(existing_table, new_data, mode="overwrite", schema_mode="overwrite") + write_deltalake( + existing_sample_table, new_data, mode="overwrite", schema_mode="overwrite" + ) - read_data = existing_table.to_pyarrow_table() + read_data = ( + QueryBuilder() + .register("tbl", existing_sample_table) + .execute("select * from tbl") + .read_all() + ) assert new_data == read_data - assert existing_table.schema().to_pyarrow() == new_data.schema +@pytest.mark.pyarrow def test_merge_schema(existing_table: DeltaTable): - print(existing_table._table.table_uri()) + import pyarrow as pa + old_table_data = existing_table.to_pyarrow_table() new_data = pa.table( { @@ -162,15 +177,25 @@ def test_merge_schema(existing_table: DeltaTable): write_deltalake(existing_table, new_data, mode="overwrite", schema_mode="overwrite") - assert existing_table.schema().to_pyarrow() == new_data.schema + assert pa.schema(existing_table.schema()) == new_data.schema +@pytest.mark.xfail(reason="update schema code to allow schemaexportable as input") def test_overwrite_schema(existing_table: DeltaTable): - new_data_invalid = pa.table( + new_data_invalid = Table( { - "utf8": pa.array([1235, 546, 5645]), - "new_x": pa.array([1, 2, 3], pa.int32()), - "new_y": pa.array([1, 2, 3], pa.int32()), + "utf8": Array( + [1235, 546, 5645], + ArrowField("utf8", type=DataType.int64(), nullable=False), + ), + "new_x": Array( + [1, 2, 3], + ArrowField("new_x", type=DataType.int32(), nullable=False), + ), + "new_y": Array( + [1, 2, 3], + ArrowField("new_y", type=DataType.int32(), nullable=False), + ), } ) @@ -182,13 +207,23 @@ def test_overwrite_schema(existing_table: DeltaTable): schema_mode="overwrite", ) - new_data = pa.table( + new_data = Table( { - "utf8": pa.array(["bla", "bli", "blubb"]), - "new_x": pa.array([1, 2, 3], pa.int32()), - "new_y": pa.array([1, 2, 3], pa.int32()), + "utf8": Array( + ["bla", "bli", "blubb"], + ArrowField("utf8", type=DataType.utf8(), nullable=False), + ), + "new_x": Array( + [1, 2, 3], + ArrowField("new_x", type=DataType.int32(), nullable=False), + ), + "new_y": Array( + [1, 2, 3], + ArrowField("new_y", type=DataType.int32(), nullable=False), + ), } ) + with pytest.raises(DeltaError): write_deltalake( existing_table, @@ -199,37 +234,66 @@ def test_overwrite_schema(existing_table: DeltaTable): write_deltalake(existing_table, new_data, mode="overwrite", schema_mode="overwrite") - assert existing_table.schema().to_pyarrow() == new_data.schema + assert existing_table.schema().to_arrow() == new_data.schema -def test_update_schema_rust_writer_append(existing_table: DeltaTable): +def test_update_schema_rust_writer_append(existing_sample_table: DeltaTable): with pytest.raises(SchemaMismatchError): # It's illegal to do schema drift without correct schema_mode write_deltalake( - existing_table, - pa.table({"x4": pa.array([1, 2, 3])}), + existing_sample_table, + Table( + { + "x4": Array( + [1, 2, 3], + ArrowField("x4", type=DataType.int64(), nullable=False), + ), + } + ), mode="append", schema_mode=None, ) with pytest.raises(DeltaError): write_deltalake( # schema_mode overwrite is illegal with append - existing_table, - pa.table({"x1": pa.array([1, 2, 3])}), + existing_sample_table, + Table( + { + "x1": Array( + [1, 2, 3], + ArrowField("x1", type=DataType.int64(), nullable=False), + ), + } + ), mode="append", schema_mode="overwrite", ) write_deltalake( - existing_table, - pa.table({"x2": pa.array([1, 2, 3])}), + existing_sample_table, + Table( + { + "x2": Array( + [1, 2, 3], + ArrowField("x2", type=DataType.int64(), nullable=False), + ), + } + ), mode="append", schema_mode="merge", ) +@pytest.mark.pyarrow def test_write_type_castable_types(existing_table: DeltaTable): write_deltalake( existing_table, - pa.table({"utf8": pa.array([1, 2, 3])}), + Table( + { + "utf8": Array( + ["1", "2", "3"], + ArrowField("utf8", type=DataType.string(), nullable=False), + ), + } + ), mode="append", schema_mode="merge", ) @@ -239,7 +303,14 @@ def test_write_type_castable_types(existing_table: DeltaTable): ): write_deltalake( existing_table, - pa.table({"int8": pa.array(["hello", "2", "3"])}), + Table( + { + "int8": Array( + ["hello", "2", "3"], + ArrowField("int8", type=DataType.string(), nullable=False), + ), + } + ), mode="append", schema_mode="merge", ) @@ -250,50 +321,90 @@ def test_write_type_castable_types(existing_table: DeltaTable): ): write_deltalake( existing_table, - pa.table({"int8": pa.array([1000, 100, 10])}), + Table( + { + "int8": Array( + [1000, 100, 10], + ArrowField("int8", type=DataType.int32(), nullable=False), + ), + } + ), mode="append", schema_mode="merge", ) -def test_update_schema_rust_writer_invalid(existing_table: DeltaTable): - new_data = pa.table({"x5": pa.array([1, 2, 3])}) +def test_update_schema_rust_writer_invalid(existing_sample_table: DeltaTable): + new_data = Table( + { + "x5": Array( + [1, 2, 3], + ArrowField("x5", type=DataType.int64(), nullable=False), + ), + } + ) + with pytest.raises( SchemaMismatchError, match="Cannot cast schema, number of fields does not match" ): write_deltalake( - existing_table, + existing_sample_table, new_data, mode="overwrite", schema_mode=None, ) write_deltalake( - existing_table, + existing_sample_table, new_data, mode="overwrite", schema_mode="overwrite", ) - read_data = existing_table.to_pyarrow_table() + read_data = ( + QueryBuilder() + .register("tbl", existing_sample_table) + .execute("select * from tbl") + .read_all() + ) assert new_data == read_data - assert existing_table.schema().to_pyarrow() == new_data.schema def test_merge_schema_rust_writer_with_overwrite(tmp_path: pathlib.Path): - data = pa.table( + data = Table( { - "a": pa.array([1, 2, 3, 4]), - "b": pa.array([1, 1, 2, 2]), - "c": pa.array([10, 11, 12, 13]), + "a": Array( + [1, 2, 3, 4], + ArrowField("a", type=DataType.int64(), nullable=True), + ), + "b": Array( + [1, 1, 2, 2], + ArrowField("b", type=DataType.int64(), nullable=True), + ), + "c": Array( + [10, 11, 12, 13], + ArrowField("c", type=DataType.int64(), nullable=True), + ), } ) + write_deltalake( tmp_path, data, ) - new_data = pa.table({"a": pa.array([100, 200, 300]), "b": pa.array([1, 1, 1])}) + new_data = Table( + { + "a": Array( + [100, 200, 300], + ArrowField("a", type=DataType.int64(), nullable=True), + ), + "b": Array( + [1, 1, 1], + ArrowField("b", type=DataType.int64(), nullable=True), + ), + } + ) write_deltalake( tmp_path, @@ -301,31 +412,41 @@ def test_merge_schema_rust_writer_with_overwrite(tmp_path: pathlib.Path): mode="overwrite", schema_mode="merge", ) - assert set(DeltaTable(tmp_path).to_pyarrow_table().column_names) == set( - ["a", "b", "c"] + dt = DeltaTable(tmp_path) + result = ( + QueryBuilder() + .register("tbl", dt) + .execute("select * from tbl") + .read_all() + .column_names ) + assert set(result) == set(["a", "b", "c"]) def test_local_path( tmp_path: pathlib.Path, - sample_data: pa.Table, + sample_table: Table, monkeypatch, ): monkeypatch.chdir(tmp_path) # Make tmp_path the working directory (tmp_path / "path/to/table").mkdir(parents=True) local_path = "./path/to/table" - write_deltalake(local_path, sample_data) + write_deltalake(local_path, sample_table) delta_table = DeltaTable(local_path) - assert delta_table.schema().to_pyarrow() == sample_data.schema - table = delta_table.to_pyarrow_table() - assert table == sample_data + table = ( + QueryBuilder() + .register("tbl", delta_table) + .execute("select * from tbl") + .read_all() + ) + assert table == sample_table def test_local_path_with_unsafe_rename( tmp_path: pathlib.Path, - sample_data: pa.Table, + sample_table: Table, monkeypatch, ): monkeypatch.chdir(tmp_path) # Make tmp_path the working directory @@ -335,18 +456,22 @@ def test_local_path_with_unsafe_rename( storage_opts = { "allow_unsafe_rename": "true", } - write_deltalake(local_path, sample_data, storage_options=storage_opts) + write_deltalake(local_path, sample_table, storage_options=storage_opts) delta_table = DeltaTable(local_path, storage_options=storage_opts) - assert delta_table.schema().to_pyarrow() == sample_data.schema - table = delta_table.to_pyarrow_table() - assert table == sample_data + table = ( + QueryBuilder() + .register("tbl", delta_table) + .execute("select * from tbl") + .read_all() + ) + assert table == sample_table -def test_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): +def test_roundtrip_metadata(tmp_path: pathlib.Path, sample_table: Table): write_deltalake( tmp_path, - sample_data, + sample_table, name="test_name", description="test_desc", configuration={"delta.appendOnly": "false"}, @@ -361,6 +486,7 @@ def test_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): assert metadata.configuration == {"delta.appendOnly": "false"} +@pytest.mark.pyarrow @pytest.mark.parametrize( "column", [ @@ -378,16 +504,19 @@ def test_roundtrip_metadata(tmp_path: pathlib.Path, sample_data: pa.Table): ], ) def test_roundtrip_partitioned( - tmp_path: pathlib.Path, sample_data: pa.Table, column: str + tmp_path: pathlib.Path, sample_data_pyarrow: "pa.Table", column: str ): - write_deltalake(tmp_path, sample_data, partition_by=column) + import pyarrow as pa + import pyarrow.compute as pc + + write_deltalake(tmp_path, sample_data_pyarrow, partition_by=column) delta_table = DeltaTable(tmp_path) - assert delta_table.schema().to_pyarrow() == sample_data.schema + assert pa.schema(delta_table.schema()) == sample_data_pyarrow.schema table = delta_table.to_pyarrow_table() table = table.take(pc.sort_indices(table["int64"])) - assert table == sample_data + assert table == sample_data_pyarrow for add_path in get_add_paths(delta_table): # Paths should be relative @@ -396,85 +525,117 @@ def test_roundtrip_partitioned( def test_roundtrip_null_partition( tmp_path: pathlib.Path, - sample_data: pa.Table, + sample_table: Table, ): - sample_data = sample_data.add_column( - 0, "utf8_with_nulls", pa.array(["a"] * 4 + [None]) + sample_table = sample_table.add_column( + 4, + "utf8_with_nulls", + ChunkedArray( + Array( + ["a", "a", "a", "a", None], + type=ArrowField("utf8_with_nulls", DataType.string(), nullable=True), + ) + ), ) + write_deltalake( tmp_path, - sample_data, + sample_table, partition_by=["utf8_with_nulls"], ) delta_table = DeltaTable(tmp_path) - assert delta_table.schema().to_pyarrow() == sample_data.schema - table = delta_table.to_pyarrow_table() - table = table.take(pc.sort_indices(table["int64"])) - assert table == sample_data + table = ( + QueryBuilder() + .register("tbl", delta_table) + .execute("select * from tbl order by price asc") + .read_all() + ) + assert table == sample_table def test_roundtrip_multi_partitioned( tmp_path: pathlib.Path, - sample_data: pa.Table, + sample_table: Table, ): - write_deltalake(tmp_path, sample_data, partition_by=["int32", "bool"]) + write_deltalake(tmp_path, sample_table, partition_by=["sold", "price"]) delta_table = DeltaTable(tmp_path) - assert delta_table.schema().to_pyarrow() == sample_data.schema - table = delta_table.to_pyarrow_table() - table = table.take(pc.sort_indices(table["int64"])) - assert table == sample_data + table = ( + QueryBuilder() + .register("tbl", delta_table) + .execute("select id, price, sold, deleted from tbl order by id asc") + .read_all() + ) + assert table == sample_table for add_path in get_add_paths(delta_table): # Paths should be relative assert add_path.count("/") == 2 -def test_write_modes(tmp_path: pathlib.Path, sample_data: pa.Table): +def test_write_modes(tmp_path: pathlib.Path, sample_table: Table): write_deltalake( tmp_path, - sample_data, + sample_table, + ) + assert ( + QueryBuilder() + .register("tbl", DeltaTable(tmp_path)) + .execute("select * from tbl") + .read_all() + == sample_table ) - assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data with pytest.raises(DeltaError): - write_deltalake(tmp_path, sample_data, mode="error") + write_deltalake(tmp_path, sample_table, mode="error") write_deltalake( tmp_path, - sample_data, + sample_table, mode="ignore", ) assert ("0" * 19 + "1.json") not in os.listdir(tmp_path / "_delta_log") write_deltalake( tmp_path, - sample_data, + sample_table, mode="append", ) - expected = pa.concat_tables([sample_data, sample_data]) - assert DeltaTable(tmp_path).to_pyarrow_table() == expected + expected = RecordBatchReader.from_batches( + sample_table.schema, [*sample_table.to_batches(), *sample_table.to_batches()] + ).read_all() + assert ( + QueryBuilder() + .register("tbl", DeltaTable(tmp_path)) + .execute("select * from tbl") + .read_all() + ) == expected write_deltalake( tmp_path, - sample_data, + sample_table, mode="overwrite", ) - assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data + assert ( + QueryBuilder() + .register("tbl", DeltaTable(tmp_path)) + .execute("select * from tbl") + .read_all() + ) == sample_table def test_append_only_should_append_only_with_the_overwrite_mode( # Create rust equivalent rust tmp_path: pathlib.Path, - sample_data: pa.Table, + sample_table: Table, ): config = {"delta.appendOnly": "true"} write_deltalake( tmp_path, - sample_data, + sample_table, mode="append", configuration=config, ) @@ -482,7 +643,7 @@ def test_append_only_should_append_only_with_the_overwrite_mode( # Create rust table = DeltaTable(tmp_path) write_deltalake( table, - sample_data, + sample_table, mode="append", ) from deltalake.exceptions import CommitFailedError @@ -493,24 +654,34 @@ def test_append_only_should_append_only_with_the_overwrite_mode( # Create rust ): write_deltalake( table, - sample_data, + sample_table, mode="overwrite", ) - expected = pa.concat_tables([sample_data, sample_data]) + expected = RecordBatchReader.from_batches( + sample_table.schema, [*sample_table.to_batches(), *sample_table.to_batches()] + ).read_all() - assert table.to_pyarrow_table() == expected + assert ( + QueryBuilder().register("tbl", table).execute("select * from tbl").read_all() + ) == expected assert table.version() == 1 -def test_writer_with_table(existing_table: DeltaTable, sample_data: pa.Table): - write_deltalake(existing_table, sample_data, mode="overwrite") - assert existing_table.to_pyarrow_table() == sample_data +def test_writer_with_table(existing_sample_table: DeltaTable, sample_table: Table): + write_deltalake(existing_sample_table, sample_table, mode="overwrite") + assert ( + QueryBuilder() + .register("tbl", existing_sample_table) + .execute("select * from tbl") + .read_all() + ) == sample_table +@pytest.mark.pyarrow def test_fails_wrong_partitioning( existing_table: DeltaTable, - sample_data: pa.Table, + sample_data_pyarrow: "pa.Table", ): with pytest.raises( DeltaError, @@ -518,32 +689,32 @@ def test_fails_wrong_partitioning( ): write_deltalake( existing_table, - sample_data, + sample_data_pyarrow, mode="append", partition_by="int32", ) @pytest.mark.pandas -@pytest.mark.parametrize("schema_provided", [True, False]) -def test_write_pandas(tmp_path: pathlib.Path, sample_data: pa.Table, schema_provided): +@pytest.mark.pyarrow +def test_write_pandas(tmp_path: pathlib.Path, sample_data_pyarrow: "pa.Table"): + from pandas.testing import assert_frame_equal + # When timestamp is converted to Pandas, it gets casted to ns resolution, # but Delta Lake schemas only support us resolution. - sample_pandas = sample_data.to_pandas() - if schema_provided is True: - schema = sample_data.schema - else: - schema = None - write_deltalake(tmp_path, sample_pandas, schema=schema) + sample_pandas = sample_data_pyarrow.to_pandas() + write_deltalake(tmp_path, sample_pandas) delta_table = DeltaTable(tmp_path) df = delta_table.to_pandas() assert_frame_equal(df, sample_pandas) @pytest.mark.pandas +@pytest.mark.pyarrow def test_to_pandas_with_types_mapper(tmp_path: pathlib.Path): """Test that DeltaTable.to_pandas() retains PyArrow Decimal type when using types_mapper.""" import pandas as pd + import pyarrow as pa schema = pa.schema( [ @@ -579,22 +750,10 @@ def types_mapper(pa_type): ) -def test_write_iterator( - tmp_path: pathlib.Path, existing_table: DeltaTable, sample_data: pa.Table -): - batches = existing_table.to_pyarrow_dataset().to_batches() - with pytest.raises(ValueError): - write_deltalake(tmp_path, batches, mode="overwrite") - - write_deltalake(tmp_path, batches, schema=sample_data.schema, mode="overwrite") - assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data - - -@pytest.mark.parametrize("large_dtypes", [True, False]) +@pytest.mark.pyarrow @pytest.mark.parametrize( "constructor", [ - lambda table: table.to_pyarrow_dataset(), lambda table: table.to_pyarrow_table(), lambda table: table.to_pyarrow_table().to_batches()[0], ], @@ -602,50 +761,51 @@ def test_write_iterator( def test_write_dataset_table_recordbatch( tmp_path: pathlib.Path, existing_table: DeltaTable, - sample_data: pa.Table, - large_dtypes: bool, + sample_data_pyarrow: "pa.Table", constructor, ): dataset = constructor(existing_table) - if large_dtypes: - dataset = convert_pyarrow_table( - sample_data, schema_conversion_mode=ArrowSchemaConversionMode.LARGE - ) write_deltalake(tmp_path, dataset, mode="overwrite") - assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data + assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data_pyarrow -@pytest.mark.parametrize("large_dtypes", [True, False]) def test_write_recordbatchreader( tmp_path: pathlib.Path, - existing_table: DeltaTable, - sample_data: pa.Table, - large_dtypes: bool, + sample_table: Table, ): - batches = existing_table.to_pyarrow_dataset().to_batches() - reader = RecordBatchReader.from_batches( - existing_table.to_pyarrow_dataset().schema, batches - ) - - if large_dtypes: - reader = convert_pyarrow_recordbatchreader( - reader, schema_conversion_mode=ArrowSchemaConversionMode.LARGE - ) - - print("writing second time") + reader = RecordBatchReader.from_arrow(sample_table) write_deltalake(tmp_path, reader, mode="overwrite") - assert DeltaTable(tmp_path).to_pyarrow_table() == sample_data + assert ( + QueryBuilder() + .register("tbl", DeltaTable(tmp_path)) + .execute("select * from tbl") + .read_all() + == sample_table + ) def test_writer_partitioning(tmp_path: pathlib.Path): - test_strings = ["a=b", "hello world", "hello%20world"] - data = pa.table( - {"p": pa.array(test_strings), "x": pa.array(range(len(test_strings)))} + table = Table( + { + "p": Array( + ["a=b", "hello world", "hello%20world"], + ArrowField("p", type=DataType.string(), nullable=False), + ), + "x": Array( + [0, 1, 2], + ArrowField("x", type=DataType.int64(), nullable=False), + ), + } ) - write_deltalake(tmp_path, data) - - assert DeltaTable(tmp_path).to_pyarrow_table() == data + write_deltalake(tmp_path, table) + result = ( + QueryBuilder() + .register("tbl", DeltaTable(tmp_path)) + .execute("select * from tbl") + .read_all() + ) + assert result == table def get_log_path(table: DeltaTable) -> str: @@ -680,10 +840,11 @@ def get_add_paths(table: DeltaTable) -> list[str]: return [action["path"] for action in get_add_actions(table)] -def test_writer_stats(existing_table: DeltaTable, sample_data: pa.Table): +@pytest.mark.pyarrow +def test_writer_stats(existing_table: DeltaTable, sample_data_pyarrow: "pa.Table"): stats = get_stats(existing_table) - assert stats["numRecords"] == sample_data.num_rows + assert stats["numRecords"] == sample_data_pyarrow.num_rows null_values = [] for null_count in stats["nullCount"].values(): @@ -734,14 +895,27 @@ def test_writer_stats(existing_table: DeltaTable, sample_data: pa.Table): def test_writer_null_stats(tmp_path: pathlib.Path): - data = pa.table( + data = Table( { - "int32": pa.array([1, None, 2, None], pa.int32()), - "float64": pa.array([1.0, None, None, None], pa.float64()), - "str": pa.array([None] * 4, pa.string()), - "bin": pa.array([b"bindata"] * 4, pa.binary()), + "int32": Array( + [1, None, 2, None], + ArrowField("int32", type=DataType.int32(), nullable=True), + ), + "float64": Array( + [1.0, None, None, None], + ArrowField("float64", type=DataType.float64(), nullable=True), + ), + "str": Array( + [None, None, None, None], + ArrowField("str", type=DataType.string(), nullable=True), + ), + "bin": Array( + [b"bindata", b"bindata", b"bindata", b"bindata"], + ArrowField("bin", type=DataType.binary(), nullable=False), + ), } ) + write_deltalake(tmp_path, data) table = DeltaTable(tmp_path) @@ -755,7 +929,15 @@ def test_try_get_table_and_table_uri(tmp_path: pathlib.Path): def _normalize_path(t): # who does not love Windows? ;) return t[0], t[1].replace("\\", "/") if t[1] else t[1] - data = pa.table({"vals": pa.array(["1", "2", "3"])}) + data = Table( + { + "vals": Array( + ["1", "2", "3"], + ArrowField("vals", type=DataType.string(), nullable=False), + ), + } + ) + table_or_uri = tmp_path / "delta_table" write_deltalake(table_or_uri, data) delta_table = DeltaTable(table_or_uri) @@ -811,39 +993,60 @@ def _normalize_path(t): # who does not love Windows? ;) try_get_table_and_table_uri(None, None) +def param_sequence(): + import pyarrow as pa + + return (1, 2, pa.int64(), "1") + + +def param_sequence2(): + import pyarrow as pa + + return (False, True, pa.bool_(), "false") + + +def param_sequence3(): + import pyarrow as pa + + return (date(2022, 1, 1), date(2022, 1, 2), pa.date32(), "'2022-01-01'") + + +@pytest.mark.pyarrow @pytest.mark.parametrize( - "value_1,value_2,value_type,filter_string", + "func", [ - (1, 2, pa.int64(), "1"), - (False, True, pa.bool_(), "false"), - (date(2022, 1, 1), date(2022, 1, 2), pa.date32(), "'2022-01-01'"), + param_sequence, + param_sequence2, + param_sequence3, ], ) def test_partition_overwrite( tmp_path: pathlib.Path, - value_1: Any, - value_2: Any, - value_type: pa.DataType, - filter_string: str, + func, ): - sample_data = pa.table( + import pyarrow as pa + + value_1, value_2, value_type, filter_string = func() + sample_data_pyarrow = pa.table( { "p1": pa.array(["1", "1", "2", "2"], pa.string()), "p2": pa.array([value_1, value_2, value_1, value_2], value_type), "val": pa.array([1, 1, 1, 1], pa.int64()), } ) - write_deltalake(tmp_path, sample_data, mode="overwrite", partition_by=["p1", "p2"]) + write_deltalake( + tmp_path, sample_data_pyarrow, mode="overwrite", partition_by=["p1", "p2"] + ) delta_table = DeltaTable(tmp_path) assert ( delta_table.to_pyarrow_table().sort_by( [("p1", "ascending"), ("p2", "ascending")] ) - == sample_data + == sample_data_pyarrow ) - sample_data = pa.table( + sample_data_pyarrow = pa.table( { "p1": pa.array(["1", "1"], pa.string()), "p2": pa.array([value_2, value_1], value_type), @@ -859,7 +1062,7 @@ def test_partition_overwrite( ) write_deltalake( tmp_path, - sample_data, + sample_data_pyarrow, mode="overwrite", predicate="p1 = 1", ) @@ -872,7 +1075,7 @@ def test_partition_overwrite( == expected_data ) - sample_data = pa.table( + sample_data_pyarrow = pa.table( { "p1": pa.array(["1", "2"], pa.string()), "p2": pa.array([value_2, value_2], value_type), @@ -889,7 +1092,7 @@ def test_partition_overwrite( write_deltalake( tmp_path, - sample_data, + sample_data_pyarrow, mode="overwrite", predicate=f"p2 > {filter_string}", ) @@ -902,7 +1105,7 @@ def test_partition_overwrite( ) # Overwrite a single partition - sample_data = pa.table( + sample_data_pyarrow = pa.table( { "p1": pa.array(["1"], pa.string()), "p2": pa.array([value_1], value_type), @@ -918,7 +1121,7 @@ def test_partition_overwrite( ) write_deltalake( tmp_path, - sample_data, + sample_data_pyarrow, mode="overwrite", predicate=f"p1 = 1 AND p2 = {filter_string}", ) @@ -931,50 +1134,59 @@ def test_partition_overwrite( ) with pytest.raises(DeltaProtocolError, match="Invariant violations"): write_deltalake( - tmp_path, sample_data, mode="overwrite", predicate=f"p2 < {filter_string}" + tmp_path, + sample_data_pyarrow, + mode="overwrite", + predicate=f"p2 < {filter_string}", ) @pytest.fixture() -def sample_data_for_partitioning() -> pa.Table: - return pa.table( +def sample_data_for_partitioning() -> Table: + return Table( { - "p1": pa.array(["1", "1", "2", "2"], pa.string()), - "p2": pa.array([1, 2, 1, 2], pa.int64()), - "val": pa.array([1, 1, 1, 1], pa.int64()), + "p1": Array( + ["1", "1", "2", "2"], + ArrowField("p1", type=DataType.string(), nullable=False), + ), + "p2": Array( + [1, 2, 1, 2], + ArrowField("p2", type=DataType.int64(), nullable=False), + ), + "val": Array( + [1, 1, 1, 1], + ArrowField("val", type=DataType.int64(), nullable=False), + ), } ) -@pytest.mark.parametrize("large_dtypes", [True, False]) +@pytest.mark.pyarrow @pytest.mark.parametrize( - "value_1,value_2,value_type,filter_string", + "func", [ - (1, 2, pa.int64(), "1"), - (False, True, pa.bool_(), "false"), - (date(2022, 1, 1), date(2022, 1, 2), pa.date32(), "2022-01-01"), + param_sequence, + param_sequence2, + param_sequence3, ], ) def test_replace_where_overwrite( tmp_path: pathlib.Path, - value_1: Any, - value_2: Any, - value_type: pa.DataType, - filter_string: str, - large_dtypes: bool, + func, ): + value_1, value_2, value_type, filter_string = func() + import pyarrow as pa + table_path = tmp_path - sample_data = pa.table( + sample_data_pyarrow = pa.table( { "p1": pa.array(["1", "1", "2", "2"], pa.string()), "p2": pa.array([value_1, value_2, value_1, value_2], value_type), "val": pa.array([1, 1, 1, 1], pa.int64()), } ) - write_data = sample_data - if large_dtypes: - write_data = convert_pyarrow_table(write_data, ArrowSchemaConversionMode.LARGE) + write_data = sample_data_pyarrow write_deltalake(table_path, write_data, mode="overwrite") @@ -983,10 +1195,10 @@ def test_replace_where_overwrite( delta_table.to_pyarrow_table().sort_by( [("p1", "ascending"), ("p2", "ascending")] ) - == sample_data + == sample_data_pyarrow ) - sample_data = pa.table( + sample_data_pyarrow = pa.table( { "p1": pa.array(["1", "1"], pa.string()), "p2": pa.array([value_2, value_1], value_type), @@ -1001,14 +1213,9 @@ def test_replace_where_overwrite( } ) - if large_dtypes: - sample_data = convert_pyarrow_table( - sample_data, ArrowSchemaConversionMode.LARGE - ) - write_deltalake( table_path, - sample_data, + sample_data_pyarrow, mode="overwrite", predicate="p1 = '1'", ) @@ -1022,24 +1229,25 @@ def test_replace_where_overwrite( ) +@pytest.mark.pyarrow @pytest.mark.parametrize( - "value_1,value_2,value_type,filter_string", + "func", [ - (1, 2, pa.int64(), "1"), - (False, True, pa.bool_(), "false"), - (date(2022, 1, 1), date(2022, 1, 2), pa.date32(), "2022-01-01"), + param_sequence, + param_sequence2, + param_sequence3, ], ) def test_replace_where_overwrite_partitioned( tmp_path: pathlib.Path, - value_1: Any, - value_2: Any, - value_type: pa.DataType, - filter_string: str, + func, ): + import pyarrow as pa + + value_1, value_2, value_type, filter_string = func() table_path = tmp_path - sample_data = pa.table( + sample_data_pyarrow = pa.table( { "p1": pa.array(["1", "1", "2", "2"], pa.string()), "p2": pa.array([value_1, value_2, value_1, value_2], value_type), @@ -1047,7 +1255,7 @@ def test_replace_where_overwrite_partitioned( } ) write_deltalake( - table_path, sample_data, mode="overwrite", partition_by=["p1", "p2"] + table_path, sample_data_pyarrow, mode="overwrite", partition_by=["p1", "p2"] ) delta_table = DeltaTable(table_path) @@ -1055,7 +1263,7 @@ def test_replace_where_overwrite_partitioned( delta_table.to_pyarrow_table().sort_by( [("p1", "ascending"), ("p2", "ascending")] ) - == sample_data + == sample_data_pyarrow ) replace_data = pa.table( @@ -1091,7 +1299,7 @@ def test_replace_where_overwrite_partitioned( def test_partition_overwrite_with_new_partition( - tmp_path: pathlib.Path, sample_data_for_partitioning: pa.Table + tmp_path: pathlib.Path, sample_data_for_partitioning: Table ): write_deltalake( tmp_path, @@ -1100,44 +1308,83 @@ def test_partition_overwrite_with_new_partition( partition_by=["p1", "p2"], ) - new_sample_data = pa.table( + new_sample_data = Table( { - "p1": pa.array(["1", "2"], pa.string()), - "p2": pa.array([2, 2], pa.int64()), - "val": pa.array([2, 3], pa.int64()), + "p1": Array( + ["1", "2"], + ArrowField("p1", type=DataType.string(), nullable=False), + ), + "p2": Array( + [2, 2], + ArrowField("p2", type=DataType.int64(), nullable=False), + ), + "val": Array( + [2, 3], + ArrowField("val", type=DataType.int64(), nullable=False), + ), } ) - expected_data = pa.table( + + expected_data = Table( { - "p1": pa.array(["1", "1", "2", "2"], pa.string()), - "p2": pa.array([1, 2, 1, 2], pa.int64()), - "val": pa.array([1, 2, 1, 3], pa.int64()), + "p1": Array( + ["1", "1", "2", "2"], + ArrowField("p1", type=DataType.string(), nullable=False), + ), + "p2": Array( + [1, 2, 1, 2], + ArrowField("p2", type=DataType.int64(), nullable=False), + ), + "val": Array( + [1, 2, 1, 3], + ArrowField("val", type=DataType.int64(), nullable=False), + ), } ) + write_deltalake(tmp_path, new_sample_data, mode="overwrite", predicate="p2 = 2") delta_table = DeltaTable(tmp_path) - assert ( - delta_table.to_pyarrow_table().sort_by( - [("p1", "ascending"), ("p2", "ascending")] - ) - == expected_data + result = ( + QueryBuilder() + .register("tbl", delta_table) + .execute("select p1,p2,val from tbl order by p1 asc, p2 asc") + .read_all() ) + assert result == expected_data def test_partition_overwrite_with_non_partitioned_data( - tmp_path: pathlib.Path, sample_data_for_partitioning: pa.Table + tmp_path: pathlib.Path, sample_data_for_partitioning: Table ): write_deltalake(tmp_path, sample_data_for_partitioning, mode="overwrite") write_deltalake( tmp_path, - sample_data_for_partitioning.filter(pc.field("p1") == "1"), + Table( + { + "p1": Array( + ["1", "1"], + ArrowField("p1", type=DataType.string(), nullable=False), + ), + "p2": Array( + [ + 1, + 2, + ], + ArrowField("p2", type=DataType.int64(), nullable=False), + ), + "val": Array( + [1, 1], + ArrowField("val", type=DataType.int64(), nullable=False), + ), + } + ), mode="overwrite", predicate="p1 = 1", ) def test_partition_overwrite_with_wrong_partition( - tmp_path: pathlib.Path, sample_data_for_partitioning: pa.Table + tmp_path: pathlib.Path, sample_data_for_partitioning: Table ): write_deltalake( tmp_path, @@ -1153,14 +1400,22 @@ def test_partition_overwrite_with_wrong_partition( sample_data_for_partitioning, mode="overwrite", predicate="p999 = 1", - # partition_filters=[("p999", "=", "1")], ) - new_data = pa.table( + new_data = Table( { - "p1": pa.array(["1"], pa.string()), - "p2": pa.array([2], pa.int64()), - "val": pa.array([1], pa.int64()), + "p1": Array( + ["1"], + ArrowField("p1", type=DataType.string(), nullable=False), + ), + "p2": Array( + [2], + ArrowField("p2", type=DataType.int64(), nullable=False), + ), + "val": Array( + [1], + ArrowField("val", type=DataType.int64(), nullable=False), + ), } ) @@ -1177,16 +1432,26 @@ def test_partition_overwrite_with_wrong_partition( def test_handles_binary_data(tmp_path: pathlib.Path): - value = b"\x00\\" - table = pa.Table.from_pydict({"field_one": [value]}) + table = Table( + { + "field_one": Array( + [b"\x00\\"], + ArrowField("field_one", type=DataType.binary(), nullable=True), + ), + } + ) + write_deltalake(tmp_path, table) dt = DeltaTable(tmp_path) - out = dt.to_pyarrow_table() + out = QueryBuilder().register("tbl", dt).execute("select * from tbl").read_all() assert table == out +@pytest.mark.pyarrow def test_large_arrow_types(tmp_path: pathlib.Path): + import pyarrow as pa + pylist = [ {"name": "Joey", "gender": b"M", "arr_type": ["x", "y"], "dict": {"a": b"M"}}, {"name": "Ivan", "gender": b"F", "arr_type": ["x", "z"]}, @@ -1195,24 +1460,29 @@ def test_large_arrow_types(tmp_path: pathlib.Path): [ pa.field("name", pa.large_string()), pa.field("gender", pa.large_binary()), - pa.field("arr_type", pa.large_list(pa.large_string())), + pa.field( + "arr_type", + pa.large_list(pa.field("element", pa.large_string(), nullable=False)), + ), pa.field("map_type", pa.map_(pa.large_string(), pa.large_binary())), pa.field("struct", pa.struct([pa.field("sub", pa.large_string())])), ] ) table = pa.Table.from_pylist(pylist, schema=schema) - table = convert_pyarrow_table(table, ArrowSchemaConversionMode.LARGE) - write_deltalake(tmp_path, table, mode="append") write_deltalake(tmp_path, table, mode="append") write_deltalake(tmp_path, table, mode="append") dt = DeltaTable(tmp_path) - assert table.schema == dt.schema().to_pyarrow(as_large_types=True) + assert table.schema == pa.schema(dt.schema().to_arrow(as_large_types=True)) +@pytest.mark.pyarrow def test_large_arrow_types_dataset_as_large_types(tmp_path: pathlib.Path): + import pyarrow as pa + from pyarrow.dataset import dataset + pylist = [ {"name": "Joey", "gender": b"M", "arr_type": ["x", "y"], "dict": {"a": b"M"}}, {"name": "Ivan", "gender": b"F", "arr_type": ["x", "z"]}, @@ -1221,7 +1491,10 @@ def test_large_arrow_types_dataset_as_large_types(tmp_path: pathlib.Path): [ pa.field("name", pa.large_string()), pa.field("gender", pa.large_binary()), - pa.field("arr_type", pa.large_list(pa.large_string())), + pa.field( + "arr_type", + pa.large_list(pa.field("element", pa.large_string(), nullable=False)), + ), pa.field("map_type", pa.map_(pa.large_string(), pa.large_binary())), pa.field("struct", pa.struct([pa.field("sub", pa.large_string())])), ] @@ -1237,7 +1510,11 @@ def test_large_arrow_types_dataset_as_large_types(tmp_path: pathlib.Path): assert union_ds.to_table().shape[0] == 4 +@pytest.mark.pyarrow def test_large_arrow_types_explicit_scan_schema(tmp_path: pathlib.Path): + import pyarrow as pa + from pyarrow.dataset import dataset + pylist = [ {"name": "Joey", "gender": b"M", "arr_type": ["x", "y"], "dict": {"a": b"M"}}, {"name": "Ivan", "gender": b"F", "arr_type": ["x", "z"]}, @@ -1263,11 +1540,20 @@ def test_large_arrow_types_explicit_scan_schema(tmp_path: pathlib.Path): def test_partition_large_arrow_types(tmp_path: pathlib.Path): - table = pa.table( + table = Table( { - "foo": pa.array(["1", "1", "2", "2"], pa.large_string()), - "bar": pa.array([1, 2, 1, 2], pa.int64()), - "baz": pa.array([1, 1, 1, 1], pa.int64()), + "foo": Array( + ["1", "1", "2", "2"], + ArrowField("foo", type=DataType.large_utf8(), nullable=True), + ), + "bar": Array( + [1, 2, 1, 2], + ArrowField("bar", type=DataType.int64(), nullable=True), + ), + "baz": Array( + [1, 1, 1, 1], + ArrowField("baz", type=DataType.int64(), nullable=True), + ), } ) @@ -1282,24 +1568,34 @@ def test_partition_large_arrow_types(tmp_path: pathlib.Path): def test_uint_arrow_types(tmp_path: pathlib.Path): - pylist = [ - {"num1": 3, "num2": 3, "num3": 3, "num4": 5}, - {"num1": 1, "num2": 13, "num3": 35, "num4": 13}, - ] - schema = pa.schema( - [ - pa.field("num1", pa.uint8()), - pa.field("num2", pa.uint16()), - pa.field("num3", pa.uint32()), - pa.field("num4", pa.uint64()), - ] + table = Table( + { + "num1": Array( + [3, 1], + ArrowField("num1", type=DataType.uint8(), nullable=True), + ), + "num2": Array( + [3, 13], + ArrowField("num2", type=DataType.uint16(), nullable=True), + ), + "num3": Array( + [3, 35], + ArrowField("num3", type=DataType.uint32(), nullable=True), + ), + "num4": Array( + [5, 13], + ArrowField("num4", type=DataType.uint64(), nullable=True), + ), + } ) - table = pa.Table.from_pylist(pylist, schema=schema) write_deltalake(tmp_path, table) +@pytest.mark.pyarrow def test_issue_1651_roundtrip_timestamp(tmp_path: pathlib.Path): + import pyarrow as pa + data = pa.table( { "id": pa.array([425], type=pa.int32()), @@ -1315,9 +1611,12 @@ def test_issue_1651_roundtrip_timestamp(tmp_path: pathlib.Path): assert dataset.count_rows() == 1 +@pytest.mark.pyarrow def test_invalid_decimals(tmp_path: pathlib.Path): import re + import pyarrow as pa + data = pa.table( {"x": pa.array([Decimal("10000000000000000000000000000000000000.0")])} ) @@ -1329,7 +1628,10 @@ def test_invalid_decimals(tmp_path: pathlib.Path): write_deltalake(table_or_uri=tmp_path, mode="append", data=data) +@pytest.mark.pyarrow def test_write_large_decimal(tmp_path: pathlib.Path): + import pyarrow as pa + data = pa.table( { "decimal_column": pa.array( @@ -1342,64 +1644,119 @@ def test_write_large_decimal(tmp_path: pathlib.Path): def test_float_values(tmp_path: pathlib.Path): - data = pa.table( + data = Table( { - "id": pa.array(range(4)), - "x1": pa.array([0.0, inf, None, 1.0]), - "x2": pa.array([0.0, -inf, None, 1.0]), + "id": Array( + [0, 1, 2, 3], + ArrowField("id", type=DataType.int64(), nullable=True), + ), + "x1": Array( + [0.0, float("inf"), None, 1.0], + ArrowField("x1", type=DataType.float64(), nullable=True), + ), + "x2": Array( + [0.0, float("-inf"), None, 1.0], + ArrowField("x2", type=DataType.float64(), nullable=True), + ), } ) + write_deltalake(tmp_path, data) dt = DeltaTable(tmp_path) - assert dt.to_pyarrow_table() == data + assert ( + QueryBuilder().register("tbl", dt).execute("select * from tbl").read_all() + == data + ) actions = dt.get_add_actions() + + def get_value(name: str): + return actions.column(name)[0].as_py() + # x1 has no max, since inf was the highest value - assert actions["min"].field("x1")[0].as_py() == -0.0 - assert actions["max"].field("x1")[0].as_py() is None - assert actions["null_count"].field("x1")[0].as_py() == 1 + assert get_value("min")["x1"] == -0.0 + assert get_value("max")["x1"] is None + assert get_value("null_count")["x1"] == 1 # x2 has no min, since -inf was the lowest value - assert actions["min"].field("x2")[0].as_py() is None - assert actions["max"].field("x2")[0].as_py() == 1.0 - assert actions["null_count"].field("x2")[0].as_py() == 1 + assert get_value("min")["x2"] is None + assert get_value("max")["x2"] == 1.0 + assert get_value("null_count")["x2"] == 1 def test_schema_cols_diff_order(tmp_path: pathlib.Path): - data = pa.table( + data = Table( { - "foo": pa.array(["B"] * 10), - "bar": pa.array([1] * 10), - "baz": pa.array([2.0] * 10), + "foo": Array( + ["B"] * 10, + ArrowField("foo", type=DataType.string(), nullable=True), + ), + "bar": Array( + [1] * 10, + ArrowField("bar", type=DataType.int64(), nullable=True), + ), + "baz": Array( + [2.0] * 10, + ArrowField("baz", type=DataType.float64(), nullable=True), + ), } ) + write_deltalake(tmp_path, data, mode="append") - data = pa.table( + data = Table( { - "baz": pa.array([2.0] * 10), - "bar": pa.array([1] * 10), - "foo": pa.array(["B"] * 10), + "baz": Array( + [2.0] * 10, + ArrowField("baz", type=DataType.float64(), nullable=True), + ), + "bar": Array( + [1] * 10, + ArrowField("bar", type=DataType.int64(), nullable=True), + ), + "foo": Array( + ["B"] * 10, + ArrowField("foo", type=DataType.string(), nullable=True), + ), } ) + write_deltalake(tmp_path, data, mode="append") dt = DeltaTable(tmp_path) assert dt.version() == 1 - expected = pa.table( + expected = Table( { - "baz": pa.array([2.0] * 20), - "bar": pa.array([1] * 20), - "foo": pa.array(["B"] * 20), + "baz": Array( + [2.0] * 20, + ArrowField("baz", type=DataType.float64(), nullable=True), + ), + "bar": Array( + [1] * 20, + ArrowField("bar", type=DataType.int64(), nullable=True), + ), + "foo": Array( + ["B"] * 20, + ArrowField("foo", type=DataType.string(), nullable=True), + ), } ) - assert dt.to_pyarrow_table(columns=["baz", "bar", "foo"]) == expected + assert ( + QueryBuilder() + .register("tbl", dt) + .execute("select baz, bar, foo from tbl") + .read_all() + .rechunk() + ) == expected +@pytest.mark.pyarrow def test_empty(existing_table: DeltaTable): - schema = existing_table.schema().to_pyarrow() + import pyarrow as pa + + schema = existing_table.schema() expected = existing_table.to_pyarrow_table() - empty_table = pa.Table.from_pylist([], schema=schema) + empty_table = pa.Table.from_pylist([], schema=pa.schema(schema)) write_deltalake( existing_table, empty_table, @@ -1411,9 +1768,12 @@ def test_empty(existing_table: DeltaTable): assert expected == existing_table.to_pyarrow_table() +@pytest.mark.pyarrow def test_rust_decimal_cast(tmp_path: pathlib.Path): import re + import pyarrow as pa + data = pa.table({"x": pa.array([Decimal("100.1")])}) write_deltalake( @@ -1458,25 +1818,37 @@ def test_rust_decimal_cast(tmp_path: pathlib.Path): def test_write_stats_column_idx(tmp_path: pathlib.Path): def _check_stats(dt: DeltaTable): add_actions_table = dt.get_add_actions(flatten=True) - stats = add_actions_table.to_pylist()[0] - - assert stats["null_count.foo"] == 2 - assert stats["min.foo"] == "a" - assert stats["max.foo"] == "b" - assert stats["null_count.bar"] == 1 - assert stats["min.bar"] == 1 - assert stats["max.bar"] == 3 - assert stats["null_count.baz"] is None - assert stats["min.baz"] is None - assert stats["max.baz"] is None - data = pa.table( + def get_value(name: str): + return add_actions_table.column(name)[0].as_py() + + assert get_value("null_count.foo") == 2 + assert get_value("min.foo") == "a" + assert get_value("max.foo") == "b" + assert get_value("null_count.bar") == 1 + assert get_value("min.bar") == 1 + assert get_value("max.bar") == 3 + assert get_value("null_count.baz") is None + assert get_value("min.baz") is None + assert get_value("max.baz") is None + + data = Table( { - "foo": pa.array(["a", "b", None, None]), - "bar": pa.array([1, 2, 3, None]), - "baz": pa.array([1, 1, None, None]), + "foo": Array( + ["a", "b", None, None], + ArrowField("foo", type=DataType.string(), nullable=True), + ), + "bar": Array( + [1, 2, 3, None], + ArrowField("bar", type=DataType.int64(), nullable=True), + ), + "baz": Array( + [1, 1, None, None], + ArrowField("baz", type=DataType.int64(), nullable=True), + ), } ) + write_deltalake( tmp_path, data, @@ -1498,25 +1870,37 @@ def _check_stats(dt: DeltaTable): def test_write_stats_columns_stats_provided(tmp_path: pathlib.Path): def _check_stats(dt: DeltaTable): add_actions_table = dt.get_add_actions(flatten=True) - stats = add_actions_table.to_pylist()[0] - - assert stats["null_count.foo"] == 2 - assert stats["min.foo"] == "a" - assert stats["max.foo"] == "b" - assert stats["null_count.bar"] is None - assert stats["min.bar"] is None - assert stats["max.bar"] is None - assert stats["null_count.baz"] == 2 - assert stats["min.baz"] == 1 - assert stats["max.baz"] == 1 - data = pa.table( + def get_value(name: str): + return add_actions_table.column(name)[0].as_py() + + assert get_value("null_count.foo") == 2 + assert get_value("min.foo") == "a" + assert get_value("max.foo") == "b" + assert get_value("null_count.bar") is None + assert get_value("min.bar") is None + assert get_value("max.bar") is None + assert get_value("null_count.baz") == 2 + assert get_value("min.baz") == 1 + assert get_value("max.baz") == 1 + + data = Table( { - "foo": pa.array(["a", "b", None, None]), - "bar": pa.array([1, 2, 3, None]), - "baz": pa.array([1, 1, None, None]), + "foo": Array( + ["a", "b", None, None], + ArrowField("foo", type=DataType.string(), nullable=True), + ), + "bar": Array( + [1, 2, 3, None], + ArrowField("bar", type=DataType.int64(), nullable=True), + ), + "baz": Array( + [1, 1, None, None], + ArrowField("baz", type=DataType.int64(), nullable=True), + ), } ) + write_deltalake( tmp_path, data, @@ -1535,17 +1919,22 @@ def _check_stats(dt: DeltaTable): _check_stats(dt) +@pytest.mark.pyarrow @pytest.mark.parametrize( "array", [ - pa.array([[datetime(2010, 1, 1)]]), - pa.array([{"foo": datetime(2010, 1, 1)}]), - pa.array([{"foo": [[datetime(2010, 1, 1)]]}]), - pa.array([{"foo": [[{"foo": datetime(2010, 1, 1)}]]}]), + lambda: __import__("pyarrow").array([[datetime(2010, 1, 1)]]), + lambda: __import__("pyarrow").array([{"foo": datetime(2010, 1, 1)}]), + lambda: __import__("pyarrow").array([{"foo": [[datetime(2010, 1, 1)]]}]), + lambda: __import__("pyarrow").array( + [{"foo": [[{"foo": datetime(2010, 1, 1)}]]}] + ), ], ) -def test_write_timestamp_ntz_nested(tmp_path: pathlib.Path, array: pa.array): - data = pa.table({"x": array}) +def test_write_timestamp_ntz_nested(tmp_path: pathlib.Path, array): + import pyarrow as pa + + data = pa.table({"x": array()}) write_deltalake( tmp_path, data, @@ -1562,19 +1951,27 @@ def test_write_timestamp_ntz_nested(tmp_path: pathlib.Path, array: pa.array): def test_parse_stats_with_new_schema(tmp_path): - sample_data = pa.table( + data = Table( { - "val": pa.array([1, 1], pa.int8()), + "val": Array( + [1, 1], + ArrowField("val", type=DataType.int8(), nullable=True), + ), } ) - write_deltalake(tmp_path, sample_data) - sample_data = pa.table( + write_deltalake(tmp_path, data) + + data = Table( { - "val": pa.array([1000000000000, 1000000000000], pa.int64()), + "val": Array( + [1000000000000, 1000000000000], + ArrowField("val", type=DataType.int64(), nullable=True), + ), } ) - write_deltalake(tmp_path, sample_data, mode="overwrite", schema_mode="overwrite") + + write_deltalake(tmp_path, data, mode="overwrite", schema_mode="overwrite") def test_roundtrip_cdc_evolution(tmp_path: pathlib.Path): @@ -1598,19 +1995,22 @@ def test_roundtrip_cdc_evolution(tmp_path: pathlib.Path): assert not os.path.isdir(os.path.join(tmp_path, "_change_data")) nrows = 5 - sample_data = pa.table( + data = Table( { - "utf8": pa.array([str(x) for x in range(nrows)]), - "int64": pa.array(list(range(nrows)), pa.int64()), - # See - # "struct": pa.array([{"x": x, "y": str(x)} for x in range(nrows)]), - # "list": pa.array([list(range(x + 1)) for x in range(nrows)]), + "utf8": Array( + [str(x) for x in range(nrows)], + ArrowField("utf8", type=DataType.utf8(), nullable=True), + ), + "int64": Array( + list(range(nrows)), + ArrowField("int64", type=DataType.int64(), nullable=True), + ), } ) write_deltalake( tmp_path, - sample_data, + data, mode="append", schema_mode="merge", ) @@ -1625,10 +2025,10 @@ def test_roundtrip_cdc_evolution(tmp_path: pathlib.Path): assert os.path.isdir(os.path.join(tmp_path, "_change_data")) -def test_empty_dataset_write(tmp_path: pathlib.Path, sample_data: pa.Table): - empty_arrow_table = sample_data.schema.empty_table() - empty_dataset = dataset(empty_arrow_table) - write_deltalake(tmp_path, empty_dataset, mode="append") +@pytest.mark.pyarrow +def test_empty_data_write(tmp_path: pathlib.Path, sample_data_pyarrow: "pa.Table"): + empty_arrow_table = sample_data_pyarrow.schema.empty_table() + write_deltalake(tmp_path, empty_arrow_table, mode="append") dt = DeltaTable(tmp_path) new_dataset = dt.to_pyarrow_dataset() @@ -1778,7 +2178,7 @@ def test_write_timestamp(tmp_path: pathlib.Path): assert protocol.min_writer_version == 2 -def test_write_transactions(tmp_path: pathlib.Path, sample_data: pa.Table): +def test_write_transactions(tmp_path: pathlib.Path, sample_table: Table): expected_transactions = [ Transaction(app_id="app_1", version=1), Transaction(app_id="app_2", version=2, last_updated=123456), @@ -1786,26 +2186,18 @@ def test_write_transactions(tmp_path: pathlib.Path, sample_data: pa.Table): commit_properties = CommitProperties(app_transactions=expected_transactions) write_deltalake( table_or_uri=tmp_path, - data=sample_data, + data=sample_table, mode="overwrite", schema_mode="overwrite", commit_properties=commit_properties, ) delta_table = DeltaTable(tmp_path) - transactions = delta_table.transaction_versions() - - assert len(transactions) == 2 + transaction_1 = delta_table.transaction_version("app_1") + assert transaction_1 == 1 - transaction_1 = transactions["app_1"] - assert transaction_1.app_id == "app_1" - assert transaction_1.version == 1 - assert transaction_1.last_updated is None - - transaction_2 = transactions["app_2"] - assert transaction_2.app_id == "app_2" - assert transaction_2.version == 2 - assert transaction_2.last_updated == 123456 + transaction_2 = delta_table.transaction_version("app_2") + assert transaction_2 == 2 @pytest.mark.polars @@ -1814,11 +2206,19 @@ def test_write_structs(tmp_path: pathlib.Path): dt = DeltaTable.create( tmp_path, - schema=pa.schema( - [ - ("a", pa.int32()), - ("b", pa.string()), - ("c", pa.struct({"d": pa.int16(), "e": pa.int16()})), + schema=Schema( + fields=[ + Field(name="a", type=PrimitiveType("integer")), + Field(name="b", type=PrimitiveType("string")), + Field( + name="c", + type=StructType( + [ + Field(name="d", type=PrimitiveType("short")), + Field(name="e", type=PrimitiveType("short")), + ] + ), + ), ] ), ) @@ -1835,18 +2235,15 @@ def test_write_structs(tmp_path: pathlib.Path): ) dt.merge( - source=df.to_arrow(), + source=df, predicate=" AND ".join([f"target.{x} = source.{x}" for x in ["a"]]), source_alias="source", target_alias="target", ).when_not_matched_insert_all().execute() - arrow_dt = dt.to_pyarrow_dataset() - new_df = pl.scan_pyarrow_dataset(arrow_dt) - new_df.collect() - @pytest.mark.polars +@pytest.mark.xfail(reason="polars needs update") def test_write_type_coercion_predicate(tmp_path: pathlib.Path): import polars as pl @@ -1861,8 +2258,10 @@ def test_write_type_coercion_predicate(tmp_path: pathlib.Path): ) +@pytest.mark.pyarrow def test_write_schema_evolved_same_metadata_id(tmp_path): """https://github.com/delta-io/delta-rs/issues/3274""" + import pyarrow as pa data_first_write = pa.array( [ @@ -1933,6 +2332,7 @@ def test_write_schema_evolved_same_metadata_id(tmp_path): # @pytest.mark.polars +@pytest.mark.xfail(reason="polars needs update") def test_write_binary_col_without_dssc(tmp_path: pathlib.Path): import polars as pl @@ -1962,6 +2362,7 @@ def test_write_binary_col_without_dssc(tmp_path: pathlib.Path): # @pytest.mark.polars +@pytest.mark.xfail(reason="polars needs update") def test_write_binary_col_with_dssc(tmp_path: pathlib.Path): import polars as pl diff --git a/python/tests/test_writerproperties.py b/python/tests/test_writerproperties.py index 55442b2098..f19a846538 100644 --- a/python/tests/test_writerproperties.py +++ b/python/tests/test_writerproperties.py @@ -1,8 +1,7 @@ import pathlib -import pyarrow as pa -import pyarrow.parquet as pq import pytest +from arro3.core import Table from deltalake import ( BloomFilterProperties, @@ -89,9 +88,12 @@ def test_invalid_fpp_value(): BloomFilterProperties(set_bloom_filter_enabled=True, fpp=1.1, ndv=30) +@pytest.mark.pyarrow def test_write_with_writerproperties( - tmp_path: pathlib.Path, sample_table: pa.Table, writer_properties: WriterProperties + tmp_path: pathlib.Path, sample_table: Table, writer_properties: WriterProperties ): + import pyarrow.parquet as pq + write_deltalake(tmp_path, sample_table, writer_properties=writer_properties) parquet_path = DeltaTable(tmp_path).file_uris()[0]